Index: lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java (revision 1391197)
+++ lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java (working copy)
@@ -76,7 +76,7 @@
* the default override this method accordingly.
*
*/
- protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException {
+ public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException {
return reader.docValues(info.name);
}
@@ -88,7 +88,7 @@
* the default override this method accordingly.
*
*/
- protected boolean canMerge(FieldInfo info) {
+ public boolean canMerge(FieldInfo info) {
return info.hasDocValues();
}
@@ -100,7 +100,7 @@
* the default override this method accordingly.
*
*/
- protected Type getDocValuesType(FieldInfo info) {
+ public Type getDocValuesType(FieldInfo info) {
return info.getDocValuesType();
}
Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java (revision 1391197)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java (working copy)
@@ -107,18 +107,18 @@
}
@Override
- protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
+ public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
throws IOException {
return reader.normValues(info.name);
}
@Override
- protected boolean canMerge(FieldInfo info) {
+ public boolean canMerge(FieldInfo info) {
return info.hasNorms();
}
@Override
- protected Type getDocValuesType(FieldInfo info) {
+ public Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
}
Index: lucene/codecs/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java
===================================================================
--- lucene/codecs/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java (revision 0)
+++ lucene/codecs/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java (working copy)
@@ -0,0 +1,142 @@
+package org.apache.lucene.codecs.filtering;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.StoredDocument;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestFilteringCodec extends LuceneTestCase {
+
+ static final BytesRef ONE_TERM = new BytesRef("one");
+
+ private static class _WriteFilter extends WriteFilter {
+
+ _WriteFilter() {}
+
+ @Override
+ public boolean addField(FieldInfo field) {
+ if (field.name.equals("w1")) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public boolean startTerm(FieldInfo info, BytesRef text) {
+ if (info.name.equals("w2") && text.equals(ONE_TERM)) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public void filterFinishTerm(TermsConsumer original, FieldInfo info, BytesRef text, TermStats stats)
+ throws IOException {
+ if (info.name.equals("w2") && text.equals(ONE_TERM)) {
+ return;
+ }
+ original.finishTerm(text, stats);
+ }
+
+ @Override
+ public List filterStoredFields(int docId, List fields) {
+ Iterator it = fields.iterator();
+ while (it.hasNext()) {
+ FilteredField ff = it.next();
+ if (ff.info.name.equals("w1")) {
+ ff.field = null;
+ }
+ }
+ return fields;
+ }
+ }
+
+ public void testBasicFiltering() throws Exception {
+ Codec codec = Codec.getDefault();
+ boolean preFlex = codec.getName().equals("Lucene3x");
+ assumeFalse("FilteringCodec doesn't work with Lucene3x codec", preFlex);
+ FilteringCodec filtering = new FilteringCodec(codec, new _WriteFilter());
+ IndexWriterConfig cfg = new IndexWriterConfig(TEST_VERSION_CURRENT,
+ new WhitespaceAnalyzer(TEST_VERSION_CURRENT));
+ cfg.setCodec(filtering);
+ Directory d = new RAMDirectory();
+ IndexWriter iw = new IndexWriter(d, cfg);
+ Document doc = new Document();
+ FieldType ft = new FieldType();
+ ft.setIndexed(true);
+ ft.setTokenized(true);
+ ft.setStored(true);
+ doc.add(new Field("w1", "throw away", ft));
+ doc.add(new Field("w2", "one two three", ft));
+ iw.addDocument(doc);
+ iw.close();
+ // XXX this is ugly ...
+ Codec.setDefault(filtering);
+ DirectoryReader r = DirectoryReader.open(d);
+ assertEquals(1, r.numDocs());
+ StoredDocument storedDoc = r.document(0);
+ assertNull("should 'throw away'", storedDoc.get("w1"));
+ assertEquals("one two three", storedDoc.get("w2"));
+ Fields fields = MultiFields.getFields(r);
+ Iterator fe = fields.iterator();
+ TermsEnum te = null;
+ while (fe.hasNext()) {
+ String field = fe.next();
+ //System.out.println("field: " + field);
+ if (!preFlex) {
+ assertFalse("w1".equals(field)); // should be filtered out completely
+ }
+ Terms terms = fields.terms(field);
+ te = terms.iterator(te);
+ BytesRef term;
+ int cnt = 0;
+ while ((term = te.next()) != null) {
+ cnt++;
+ if ("w2".equals(field)) {
+ assertFalse(term.equals(ONE_TERM));
+ }
+ }
+ if (preFlex && "w1".equals(field)) {
+ assertEquals(0, cnt); // should be no terms
+ }
+ }
+ }
+
+}
Property changes on: lucene/codecs/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java (revision 1391197)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java (working copy)
@@ -100,18 +100,18 @@
}
@Override
- protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
+ public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
throws IOException {
return reader.normValues(info.name);
}
@Override
- protected boolean canMerge(FieldInfo info) {
+ public boolean canMerge(FieldInfo info) {
return info.hasNorms();
}
@Override
- protected Type getDocValuesType(FieldInfo info) {
+ public Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java (working copy)
@@ -0,0 +1,249 @@
+package org.apache.lucene.codecs.filtering;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfoPerCommit;
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.MutableBits;
+
+/**
+ * This class implements a strategy for filtering. Individual methods are applied
+ * in different contexts and from different threads, so care should be taken that
+ * subclasses are thread-safe.
+ */
+public class WriteFilter {
+
+ // fields
+ /**
+ * Decide whether to add a postings for a field identified by this {@link FieldInfo}.
+ * Default implementation always returns true.
+ * @param field field info
+ * @return true if this field's postings should be added, false otherwise.
+ */
+ public boolean addField(final FieldInfo field) {
+ return true;
+ }
+
+ // terms
+ /**
+ * Decide whether to add postings for this term. Returns true by default.
+ * IMPORTANT: if this method returns false
+ * then you MUST override {@link #filterFinishTerm(TermsConsumer, FieldInfo, BytesRef, TermStats)}
+ * and you MUST NOT call {@link TermsConsumer#finishTerm(BytesRef, TermStats)} there for this term.
+ * @param info field info for this term
+ * @param text term
+ * @return if true then add postings for this term, if false then ignore
+ * postings for this term (see the note above!)
+ */
+ public boolean startTerm(final FieldInfo info, final BytesRef text) {
+ return true;
+ }
+
+ /**
+ * Finish adding postings for a term. See notes for {@link #startTerm(FieldInfo, BytesRef)}.
+ * @param original original {@link TermsConsumer}. If {@link #startTerm(FieldInfo, BytesRef)}
+ * returned true for this term, then you should call {@link TermsConsumer#finishTerm(BytesRef, TermStats)}
+ * using this instance (this is the default implementation).
+ * @param info field info for this term
+ * @param text term
+ * @param stats term statistics
+ */
+ public void filterFinishTerm(final TermsConsumer original, final FieldInfo info, final BytesRef text, TermStats stats)
+ throws IOException {
+ original.finishTerm(text, stats);
+ }
+
+ /**
+ * This method is called to finish adding all postings for all terms for a
+ * given field. You may adjust overall statistics in this call before
+ * calling {@link TermsConsumer#finish(long, long, int)} on the original
+ * instance.
+ * @param original original {@link TermsConsumer}
+ * @param info field info
+ * @param sumTotalTermFreq total sum of all term frequencies across all documents
+ * @param sumDocFreq sum of document frequencies for all terms
+ * @param docCount number of documents with this field
+ */
+ public void filterFinishTerms(final TermsConsumer original, final FieldInfo info, long sumTotalTermFreq,
+ long sumDocFreq, int docCount) throws IOException {
+ original.finish(sumTotalTermFreq, sumDocFreq, docCount);
+ }
+
+ /**
+ * Decide whether to add postings for a particular document. By default it
+ * calls the original {@link PostingsConsumer#startDoc(int, int)}. Note that
+ * if you already skipped postings for this field in
+ * {@link #startTerm(FieldInfo, BytesRef)} then this method won't be called
+ * for this term.
+ *
+ * IMPORTANT: if this method doesn't call the original then you MUST
+ * override
+ * {@link #filterFinishDoc(PostingsConsumer, FieldInfo, BytesRef, int)} and
+ * you MUST NOT call the original {@link PostingsConsumer#finishDoc()} for the
+ * same term.
+ *
+ * @param original
+ * original {@link PostingsConsumer}
+ * @param info
+ * field info
+ * @param text
+ * term
+ * @param docID
+ * internal document id for which the postings are to follow
+ * @param termDocFreq
+ * original term document frequency - if some postings will be
+ * skipped then you should adjust this number accordingly.
+ * @return true if postings are added (the original
+ * {@link PostingsConsumer#startDoc(int, int)} has been called), false
+ * otherwise. It's important to return correct value here as this is
+ * used for internal consistency checks in many codecs.
+ */
+ public boolean filterStartDoc(final PostingsConsumer original, final FieldInfo info, final BytesRef text, int docID,
+ int termDocFreq) throws IOException {
+ original.startDoc(docID, termDocFreq);
+ return true;
+ }
+
+ /**
+ * Finish adding term's postings for a document. By default it calls the original
+ * {@link PostingsConsumer#finishDoc()}. IMPORTANT: see note in
+ * {@link #filterStartDoc(PostingsConsumer, FieldInfo, BytesRef, int, int)}!
+ * @param original original {@link PostingsConsumer}
+ * @param info field info
+ * @param text term
+ * @param docID internal doc id for which to finish add postings
+ */
+ public void filterFinishDoc(final PostingsConsumer original, final FieldInfo info, final BytesRef text, int docID)
+ throws IOException {
+ original.finishDoc();
+ }
+
+ /**
+ * Filter position data. Default implementation calls
+ * {@link PostingsConsumer#addPosition(int, BytesRef, int, int)} using provided
+ * arguments. You may safely skip calling this method, or adjust the arguments.
+ * @param original original {@link PostingsConsumer}
+ * @param info field info
+ * @param text current term
+ * @param docID current internal doc id
+ * @param position position
+ * @param payload payload data
+ * @param startOffset start offset
+ * @param endOffset end offset
+ */
+ public void filterAddPosition(final PostingsConsumer original, final FieldInfo info, final BytesRef text,
+ final int docID, int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
+ original.addPosition(position, payload, startOffset, endOffset);
+ }
+
+ // -------- FieldInfos ----------
+ /**
+ * Modify {@link FieldInfos}. You can e.g. skip fields, modify their type,
+ * drop/create norms, drop/create doc values, etc.
+ * Default implementation passes the original unchanged.
+ * @param directory directory
+ * @param segmentName name of the current segment being written
+ * @param infos original field infos
+ * @return possibly modified field infos
+ */
+ public FieldInfos filterFieldInfos(final Directory directory, final String segmentName, FieldInfos infos) {
+ return infos;
+ }
+
+ // -------- DocValues -----------
+
+ /**
+ * Decide whether to add a doc value for a particular doc. Returns true by default.
+ * @param docID document id
+ * @param value doc value
+ * @param norms true if the value represents Lucene norms
+ * @return true if the value should be added, false otherwise.
+ */
+ public boolean addDocValue(final int docID, final StorableField value, final boolean norms) {
+ return true;
+ }
+
+ /**
+ * Filter or modify doc value for a doc.
+ * @param docID document id
+ * @param value original value
+ * @param norms true if the value represents Lucene norms
+ * @return the original or modified value. Must never return null.
+ * @see #addDocValue(int, StorableField, boolean)
+ */
+ public StorableField filterDocValue(final int docID, StorableField value, final boolean norms) {
+ return value;
+ }
+
+ // -------- LiveDocs ----------
+ /**
+ * Filter live documents. Default implementation returns the original live docs
+ * bitset.
+ * @param bits original bit set
+ * @param dir directory of the index
+ * @param info segment info
+ * @return original or possibly modified bitset. May return null to indicate
+ * that all docs are live.
+ */
+ public MutableBits filterLiveDocs(MutableBits bits, final Directory dir, SegmentInfoPerCommit info) {
+ return bits;
+ }
+
+ // StoredFieldsWriter
+ /**
+ * Filter stored fields for a document. You can modify values of fields or
+ * drop them altogether, so long as they match the original FieldInfo-s. Default
+ * implementation simply returns the original list.
+ * @param docId current document id
+ * @param original original list of stored field values and corresponding field infos
+ * @return possibly modified list of field values and corresponding field infos, or null
+ * to drop all stored fields for this document.
+ */
+ public List filterStoredFields(int docId, List original) {
+ return original;
+ }
+
+ /**
+ * Simple container for a pair of field info and stored field value.
+ */
+ public static final class FilteredField {
+ public final FieldInfo info;
+ public StorableField field;
+
+ public FilteredField(final FieldInfo info, StorableField field) {
+ this.info = info;
+ this.field = field;
+ }
+
+ @Override
+ public String toString() {
+ return info.toString() + ":" + field.toString();
+ }
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java (working copy)
@@ -0,0 +1,57 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.StoredFieldsReader;
+import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+public class FilteringStoredFieldsFormat extends StoredFieldsFormat {
+
+ private final StoredFieldsFormat original;
+ private final WriteFilter writeFilter;
+
+ public FilteringStoredFieldsFormat(StoredFieldsFormat original, WriteFilter writeFilter) {
+ this.original = original;
+ this.writeFilter = writeFilter;
+ }
+
+ @Override
+ public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context)
+ throws IOException {
+ return original.fieldsReader(directory, si, fn, context);
+ }
+
+ @Override
+ public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo info, IOContext context) throws IOException {
+ StoredFieldsWriter writer = original.fieldsWriter(directory, info, context);
+ if (writeFilter == null) {
+ return writer;
+ } else {
+ return new FilteringStoredFieldsWriter(writer, writeFilter);
+ }
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java (working copy)
@@ -0,0 +1,90 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Comparator;
+
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.util.BytesRef;
+
+public class FilteringTermsConsumer extends TermsConsumer {
+
+ private final WriteFilter writeFilter;
+ private final TermsConsumer original;
+ private final FieldInfo info;
+
+ private BytesRef lastStartTerm = null;
+ private FilteringPostingsConsumer fpc;
+
+ public FilteringTermsConsumer(TermsConsumer original, FieldInfo info, WriteFilter filter) throws IOException {
+ this.writeFilter = filter;
+ this.original = original;
+ this.info = info;
+ }
+
+ // Called from asserts
+ private boolean storeLastStartTerm(final BytesRef text) {
+ lastStartTerm = text.clone();
+ return true;
+ }
+
+ // Called from asserts
+ private boolean verifyLastStartTerm(final BytesRef text) {
+ assert lastStartTerm != null;
+ assert lastStartTerm.equals(text);
+ return true;
+ }
+
+ @Override
+ public PostingsConsumer startTerm(final BytesRef text) throws IOException {
+ assert storeLastStartTerm(text);
+ if (!writeFilter.startTerm(info, text)) {
+ return NoopPostingsConsumer.INSTANCE;
+ }
+ fpc = new FilteringPostingsConsumer(original, info, text, writeFilter);
+ return fpc;
+ }
+
+ @Override
+ public void finishTerm(final BytesRef text, TermStats stats) throws IOException {
+ assert verifyLastStartTerm(text);
+ if (fpc != null && fpc.numDocsStarted != stats.docFreq) {
+ stats = new TermStats(fpc.numDocsStarted, fpc.totalTermFreq);
+ }
+ if (stats.docFreq == 0) {
+ return;
+ }
+ writeFilter.filterFinishTerm(original, info, text, stats);
+ }
+
+ @Override
+ public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
+ writeFilter.filterFinishTerms(original, info, sumTotalTermFreq, sumDocFreq, docCount);
+ }
+
+ @Override
+ public Comparator getComparator() throws IOException {
+ return original.getComparator();
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java (working copy)
@@ -0,0 +1,54 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.codecs.PerDocProducer;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.PerDocWriteState;
+import org.apache.lucene.index.SegmentReadState;
+
+public class FilteringNormsFormat extends NormsFormat {
+
+ private final NormsFormat original;
+ private final WriteFilter writeFilter;
+
+ public FilteringNormsFormat(NormsFormat original, WriteFilter writeFilter) {
+ this.original = original;
+ this.writeFilter = writeFilter;
+ }
+
+ @Override
+ public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
+ if (writeFilter == null) {
+ return original.docsConsumer(state);
+ } else {
+ PerDocConsumer consumer = original.docsConsumer(state);
+ return new FilteringPerDocConsumer(consumer, true, writeFilter);
+ }
+ }
+
+ @Override
+ public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
+ return original.docsProducer(state);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfoWriter.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfoWriter.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfoWriter.java (working copy)
@@ -0,0 +1,41 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.SegmentInfoWriter;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+public class FilteringSegmentInfoWriter extends SegmentInfoWriter {
+
+ private final SegmentInfoWriter original;
+
+ public FilteringSegmentInfoWriter(SegmentInfoWriter original) {
+ this.original = original;
+ }
+
+ @Override
+ public void write(Directory dir, SegmentInfo info, FieldInfos fis, IOContext context) throws IOException {
+ original.write(dir, info, fis, context);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfoWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java (working copy)
@@ -0,0 +1,44 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.FieldInfosWriter;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+class FilteringFieldInfosWriter extends FieldInfosWriter {
+
+ private final FieldInfosWriter original;
+ private final WriteFilter writeFilter;
+
+ FilteringFieldInfosWriter(FieldInfosWriter original, WriteFilter writeFilter) throws IOException {
+ this.original = original;
+ this.writeFilter = writeFilter;
+ }
+
+ @Override
+ public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
+ FieldInfos filteredInfos = writeFilter.filterFieldInfos(directory, segmentName, infos);
+ original.write(directory, segmentName, filteredInfos, context);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java (working copy)
@@ -0,0 +1,55 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+public class FilteringPostingsFormat extends PostingsFormat {
+
+ private final PostingsFormat original;
+ private final WriteFilter writeFilter;
+
+ public FilteringPostingsFormat(PostingsFormat original, WriteFilter writeFilter) {
+ super(original.getName());
+ this.original = original;
+ this.writeFilter = writeFilter;
+ }
+
+ @Override
+ public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+ if (writeFilter == null) {
+ return original.fieldsConsumer(state);
+ } else {
+ FieldsConsumer consumer = original.fieldsConsumer(state);
+ return new FilteringFieldsConsumer(consumer, writeFilter);
+ }
+ }
+
+ @Override
+ public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+ return original.fieldsProducer(state);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java (working copy)
@@ -0,0 +1,72 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.SegmentInfoPerCommit;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.MutableBits;
+
+public class FilteringLiveDocsFormat extends LiveDocsFormat {
+
+ private final LiveDocsFormat original;
+ private final WriteFilter writeFilter;
+
+ public FilteringLiveDocsFormat(LiveDocsFormat original, WriteFilter writeFilter) {
+ this.original = original;
+ this.writeFilter = writeFilter;
+ }
+
+ @Override
+ public MutableBits newLiveDocs(int size) throws IOException {
+ return original.newLiveDocs(size);
+ }
+
+ @Override
+ public MutableBits newLiveDocs(Bits existing) throws IOException {
+ return original.newLiveDocs(existing);
+ }
+
+ @Override
+ public Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) throws IOException {
+ return original.readLiveDocs(dir, info, context);
+ }
+
+ @Override
+ public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info,
+ int newDelCount, IOContext context) throws IOException {
+ if (writeFilter == null) {
+ original.writeLiveDocs(bits, dir, info, newDelCount, context);
+ } else {
+ MutableBits filteredBits = writeFilter.filterLiveDocs(bits, dir, info);
+ original.writeLiveDocs(filteredBits, dir, info, newDelCount, context);
+ }
+ }
+
+ @Override
+ public void files(SegmentInfoPerCommit info, Collection files) throws IOException {
+ original.files(info, files);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java (working copy)
@@ -0,0 +1,56 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.FieldInfo;
+
+class FilteringFieldsConsumer extends FieldsConsumer {
+
+ private final FieldsConsumer original;
+ private final WriteFilter writeFilter;
+
+ FilteringFieldsConsumer(FieldsConsumer original, WriteFilter filter) {
+ this.writeFilter = filter;
+ this.original = original;
+ }
+
+ @Override
+ public TermsConsumer addField(FieldInfo field) throws IOException {
+ if (writeFilter == null) {
+ return original.addField(field);
+ } else {
+ if (writeFilter.addField(field)) {
+ TermsConsumer consumer = original.addField(field);
+ return new FilteringTermsConsumer(consumer, field, writeFilter);
+ } else {
+ return NoopTermsConsumer.INSTANCE;
+ }
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ original.close();
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java (working copy)
@@ -0,0 +1,73 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.util.BytesRef;
+
+public class FilteringPostingsConsumer extends PostingsConsumer {
+
+ private final WriteFilter writeFilter;
+ private final FieldInfo info;
+ private final BytesRef text;
+ private final TermsConsumer originalTermsConsumer;
+
+ private PostingsConsumer original = null;
+ private int curDocID;
+ int numDocsStarted;
+ long totalTermFreq;
+
+ public FilteringPostingsConsumer(TermsConsumer originalTermsConsumer, FieldInfo info, BytesRef text,
+ WriteFilter writeFilter) throws IOException {
+ this.writeFilter = writeFilter;
+ this.info = info;
+ this.text = text;
+ this.originalTermsConsumer = originalTermsConsumer;
+ numDocsStarted = 0;
+ totalTermFreq = 0;
+ }
+
+ @Override
+ public void startDoc(int docID, int termDocFreq) throws IOException {
+ // create lazily
+ if (original == null) {
+ original = originalTermsConsumer.startTerm(text);
+ }
+ curDocID = docID;
+ if (writeFilter.filterStartDoc(original, info, text, docID, termDocFreq)) {
+ numDocsStarted++;
+ totalTermFreq += termDocFreq;
+ }
+ }
+
+ @Override
+ public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
+ writeFilter.filterAddPosition(original, info, text, curDocID, position, payload, startOffset, endOffset);
+ }
+
+ @Override
+ public void finishDoc() throws IOException {
+ writeFilter.filterFinishDoc(original, info, text, curDocID);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java (working copy)
@@ -0,0 +1,105 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.codecs.filtering.WriteFilter.FilteredField;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.StorableField;
+
+public class FilteringStoredFieldsWriter extends StoredFieldsWriter {
+
+ private final WriteFilter writeFilter;
+ private final StoredFieldsWriter original;
+
+ private List fields = null;
+ private int docNum;
+ private int numDocs;
+
+ public FilteringStoredFieldsWriter(StoredFieldsWriter original, WriteFilter writeFilter) {
+ this.writeFilter = writeFilter;
+ this.original = original;
+ docNum = 0;
+ numDocs = 0;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (fields != null) {
+ flush();
+ }
+ original.close();
+ }
+
+ @Override
+ public void startDocument(int numStoredFields) throws IOException {
+ if (fields != null) {
+ flush();
+ }
+ fields = new ArrayList(numStoredFields);
+ }
+
+ @Override
+ public void writeField(FieldInfo info, StorableField field) throws IOException {
+ fields.add(new FilteredField(info, field));
+ }
+
+ @Override
+ public void abort() {
+ original.abort();
+ fields = null;
+ }
+
+ private void flush() throws IOException {
+ List filteredFields = writeFilter.filterStoredFields(docNum, fields);
+ docNum++;
+ if (filteredFields == null || filteredFields.isEmpty()) {
+ //System.out.println(" - drop " + (docNum - 1) + " " + fields);
+ original.startDocument(0);
+ fields = null;
+ return;
+ }
+ List outputFields = new ArrayList(filteredFields.size());
+ for (FilteredField ff : filteredFields) {
+ if (ff != null && ff.field != null) {
+ outputFields.add(ff);
+ }
+ }
+ original.startDocument(outputFields.size());
+ for (FilteredField ff : outputFields) {
+ original.writeField(ff.info, ff.field);
+ }
+ numDocs++;
+ fields = null;
+ }
+
+ @Override
+ public void finish(FieldInfos infos, int numDocs) throws IOException {
+ if (fields != null) {
+ flush();
+ }
+ original.finish(infos, numDocs);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java (working copy)
@@ -0,0 +1,53 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Comparator;
+
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.util.BytesRef;
+
+public class NoopTermsConsumer extends TermsConsumer {
+
+ public static final NoopTermsConsumer INSTANCE = new NoopTermsConsumer();
+
+ @Override
+ public PostingsConsumer startTerm(BytesRef text) throws IOException {
+ return NoopPostingsConsumer.INSTANCE;
+ }
+
+ @Override
+ public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+ // noop
+ }
+
+ @Override
+ public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)
+ throws IOException {
+ // noop
+ }
+
+ @Override
+ public Comparator getComparator() throws IOException {
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java (working copy)
@@ -0,0 +1,54 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.codecs.PerDocProducer;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.PerDocWriteState;
+import org.apache.lucene.index.SegmentReadState;
+
+public class FilteringDocValuesFormat extends DocValuesFormat {
+
+ private final DocValuesFormat original;
+ private final WriteFilter writeFilter;
+
+ public FilteringDocValuesFormat(DocValuesFormat original, WriteFilter writeFilter) {
+ this.original = original;
+ this.writeFilter = writeFilter;
+ }
+
+ @Override
+ public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
+ if (writeFilter == null) {
+ return original.docsConsumer(state);
+ } else {
+ PerDocConsumer consumer = original.docsConsumer(state);
+ return new FilteringPerDocConsumer(consumer, false, writeFilter);
+ }
+ }
+
+ @Override
+ public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
+ return original.docsProducer(state);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java (working copy)
@@ -0,0 +1,65 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.index.StorableField;
+
+public class FilteringDocValuesConsumer extends DocValuesConsumer {
+
+ private final DocValuesConsumer original;
+ private final WriteFilter writeFilter;
+ private final boolean norms;
+ private final Type type;
+
+ FilteringDocValuesConsumer(DocValuesConsumer original, Type type, boolean norms, WriteFilter writeFilter)
+ throws IOException {
+ this.original = original;
+ this.norms = norms;
+ this.type = type;
+ this.writeFilter = writeFilter;
+ }
+
+ @Override
+ public void add(int docID, StorableField value) throws IOException {
+ if (writeFilter.addDocValue(docID, value, norms)) {
+ StorableField val = writeFilter.filterDocValue(docID, value, norms);
+ original.add(docID, val);
+ }
+ }
+
+ @Override
+ public void finish(int docCount) throws IOException {
+ original.finish(docCount);
+ }
+
+ @Override
+ protected Type getType() {
+ return type;
+ }
+
+ @Override
+ public int getValueSize() {
+ return original.getValueSize();
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java (working copy)
@@ -0,0 +1,45 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.util.BytesRef;
+
+public class NoopPostingsConsumer extends PostingsConsumer {
+
+ public static final NoopPostingsConsumer INSTANCE = new NoopPostingsConsumer();
+
+ @Override
+ public void startDoc(int docID, int termDocFreq) throws IOException {
+ // noop
+ }
+
+ @Override
+ public void addPosition(int position, BytesRef payload, int startOffset,
+ int endOffset) throws IOException {
+ // noop
+ }
+
+ @Override
+ public void finishDoc() throws IOException {
+ // noop
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java (working copy)
@@ -0,0 +1,49 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+public class FilteringTermVectorsFormat extends TermVectorsFormat {
+
+ private final TermVectorsFormat original;
+
+ public FilteringTermVectorsFormat(TermVectorsFormat original) {
+ this.original = original;
+ }
+
+ @Override
+ public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
+ IOContext context) throws IOException {
+ return original.vectorsReader(directory, segmentInfo, fieldInfos, context);
+ }
+
+ @Override
+ public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo info, IOContext context) throws IOException {
+ return original.vectorsWriter(directory, info, context);
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfoFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfoFormat.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfoFormat.java (working copy)
@@ -0,0 +1,42 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.SegmentInfoReader;
+import org.apache.lucene.codecs.SegmentInfoWriter;
+
+public class FilteringSegmentInfoFormat extends SegmentInfoFormat {
+
+ private final SegmentInfoFormat original;
+
+ public FilteringSegmentInfoFormat(SegmentInfoFormat original) {
+ this.original = original;
+ }
+
+ @Override
+ public SegmentInfoReader getSegmentInfoReader() {
+ return original.getSegmentInfoReader();
+ }
+
+ @Override
+ public SegmentInfoWriter getSegmentInfoWriter() {
+ return new FilteringSegmentInfoWriter(original.getSegmentInfoWriter());
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfoFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java (working copy)
@@ -0,0 +1,63 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.index.FieldInfo;
+
+class FilteringPerDocConsumer extends PerDocConsumer {
+
+ private final WriteFilter writeFilter;
+ private final PerDocConsumer original;
+ private final boolean norms;
+
+ FilteringPerDocConsumer(PerDocConsumer original, boolean norms, WriteFilter writeFilter) {
+ this.writeFilter = writeFilter;
+ this.original = original;
+ this.norms = norms;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (original != null) {
+ original.close();
+ }
+ }
+
+ @Override
+ public DocValuesConsumer addValuesField(Type type, FieldInfo field) throws IOException {
+ if (original != null) {
+ return new FilteringDocValuesConsumer(original.addValuesField(type, field), type, norms, writeFilter);
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public void abort() {
+ if (original != null) {
+ original.abort();
+ }
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java (working copy)
@@ -0,0 +1,50 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.index.DocValues.Type;
+
+class NoopDocValuesConsumer extends DocValuesConsumer {
+
+ static final DocValuesConsumer INSTANCE = new NoopDocValuesConsumer();
+
+ @Override
+ public void add(int docID, StorableField value) throws IOException {
+ // no-op
+ }
+
+ @Override
+ public void finish(int docCount) throws IOException {
+ // no-op
+ }
+
+ @Override
+ protected Type getType() {
+ return Type.FLOAT_32;
+ }
+
+ @Override
+ public int getValueSize() {
+ return -1;
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java (working copy)
@@ -0,0 +1,51 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FieldInfosReader;
+import org.apache.lucene.codecs.FieldInfosWriter;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+
+public class FilteringFieldInfosFormat extends FieldInfosFormat {
+
+ private final FieldInfosFormat original;
+ private final WriteFilter writeFilter;
+
+ public FilteringFieldInfosFormat(FieldInfosFormat original, WriteFilter writeFilter) {
+ this.original = original;
+ this.writeFilter = writeFilter;
+ }
+
+ @Override
+ public FieldInfosReader getFieldInfosReader() throws IOException {
+ return original.getFieldInfosReader();
+ }
+
+ @Override
+ public FieldInfosWriter getFieldInfosWriter() throws IOException {
+ if (writeFilter == null) {
+ return original.getFieldInfosWriter();
+ } else {
+ return new FilteringFieldInfosWriter(original.getFieldInfosWriter(), writeFilter);
+ }
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property
Index: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java
===================================================================
--- lucene/codecs/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java (revision 0)
+++ lucene/codecs/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java (working copy)
@@ -0,0 +1,116 @@
+package org.apache.lucene.codecs.filtering;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringDocValuesFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringFieldInfosFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringLiveDocsFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringNormsFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringPostingsFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringSegmentInfoFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringStoredFieldsFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringTermVectorsFormat;
+
+/**
+ * This codec allows filtering of data written to the index. It doesn't change
+ * the data encoding (it uses the original codec, and records the original
+ * codec's name) but it allows to skip or modify on the fly some of the data.
+ * The details of this filtering process are determined by a subclass of
+ * {@link WriteFilter}, which by default passes all data transparently.
+ */
+public class FilteringCodec extends Codec {
+
+ private final FilteringPostingsFormat postingsFormat;
+ private final FilteringDocValuesFormat docValuesFormat;
+ private final FilteringStoredFieldsFormat storedFieldsFormat;
+ private final FilteringTermVectorsFormat termVectorsFormat;
+ private final FilteringFieldInfosFormat fieldInfosFormat;
+ private final FilteringSegmentInfoFormat segmentInfoFormat;
+ private final FilteringNormsFormat normsFormat;
+ private final FilteringLiveDocsFormat liveDocsFormat;
+
+ /**
+ * Create a filtering codec.
+ *
+ * @param original
+ * original codec used to encode the data.
+ * @param writeFilter
+ * subclass of {@link WriteFilter}, or null for no filtering.
+ */
+ public FilteringCodec(Codec original, WriteFilter writeFilter) {
+ super(original.getName());
+ postingsFormat = new FilteringPostingsFormat(original.postingsFormat(), writeFilter);
+ docValuesFormat = new FilteringDocValuesFormat(original.docValuesFormat(), writeFilter);
+ storedFieldsFormat = new FilteringStoredFieldsFormat(original.storedFieldsFormat(), writeFilter);
+ termVectorsFormat = new FilteringTermVectorsFormat(original.termVectorsFormat());
+ fieldInfosFormat = new FilteringFieldInfosFormat(original.fieldInfosFormat(), writeFilter);
+ segmentInfoFormat = new FilteringSegmentInfoFormat(original.segmentInfoFormat());
+ normsFormat = new FilteringNormsFormat(original.normsFormat(), writeFilter);
+ liveDocsFormat = new FilteringLiveDocsFormat(original.liveDocsFormat(), writeFilter);
+ }
+
+ @Override
+ public PostingsFormat postingsFormat() {
+ return postingsFormat;
+ }
+
+ @Override
+ public DocValuesFormat docValuesFormat() {
+ return docValuesFormat;
+ }
+
+ @Override
+ public StoredFieldsFormat storedFieldsFormat() {
+ return storedFieldsFormat;
+ }
+
+ @Override
+ public TermVectorsFormat termVectorsFormat() {
+ return termVectorsFormat;
+ }
+
+ @Override
+ public FieldInfosFormat fieldInfosFormat() {
+ return fieldInfosFormat;
+ }
+
+ @Override
+ public SegmentInfoFormat segmentInfoFormat() {
+ return segmentInfoFormat;
+ }
+
+ @Override
+ public NormsFormat normsFormat() {
+ return normsFormat;
+ }
+
+ @Override
+ public LiveDocsFormat liveDocsFormat() {
+ return liveDocsFormat;
+ }
+
+}
Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
\ No newline at end of property