Index: lucene/misc/src/test/org/apache/lucene/index/TestTeeIndexSplitter.java =================================================================== --- lucene/misc/src/test/org/apache/lucene/index/TestTeeIndexSplitter.java (revision 0) +++ lucene/misc/src/test/org/apache/lucene/index/TestTeeIndexSplitter.java (working copy) @@ -0,0 +1,162 @@ +package org.apache.lucene.index; +/** + * 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.util.BitSet; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.StringField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.RAMDirectory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.junit.Test; + +public class TestTeeIndexSplitter extends LuceneTestCase { + Directory dir; + int NUM_DOCS = 100; + int SPLITS = 10; + int DELTA = 10; + + @Override + public void setUp() throws Exception { + super.setUp(); + dir = newDirectory(); + IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( + TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(NoMergePolicy.NO_COMPOUND_FILES)); + Document doc; + for (int i = 0; i < NUM_DOCS; i++) { + doc = new Document(); + doc.add(newField("id", i + "", StringField.TYPE_STORED)); + doc.add(newField("f", i + " " + i, TextField.TYPE_STORED)); + w.addDocument(doc); + if (i%3==0) w.commit(); + } + w.commit(); + w.deleteDocuments(new Term("id", "" + (NUM_DOCS-1))); + w.close(); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + dir.close(); + } + + @Test + public void testSplitter() throws Exception { + assumeFalse("TeeCodec doesn't work with Lucene3x codec", + "Lucene3x".equals(Codec.getDefault().getName())); + DirectoryReader main = DirectoryReader.open(dir); + Directory merged = newDirectory(); + Map liveDocsMap = new HashMap(); + for (int i = 0; i < SPLITS; i++) { + BitSet b = new BitSet(main.maxDoc()); + Directory d = new RAMDirectory(); + for (int j = i * DELTA; j < (i + 1) * DELTA; j++) { + b.set(j); + } + // set doc 11 for all outputs + b.set(11); + liveDocsMap.put(d, b); + } + TeeIndexSplitter.split(main, merged, liveDocsMap); + main.close(); + // check the merged output + DirectoryReader mergedReader = DirectoryReader.open(merged); + assertEquals("should have " + (NUM_DOCS-1) + " docs", NUM_DOCS-1, mergedReader.numDocs()); + assertEquals("should not have deleted docs", mergedReader.maxDoc(), mergedReader.numDocs()); + for (int i = 0; i < mergedReader.maxDoc(); i++) { + StoredDocument d = mergedReader.document(i); + StorableField id = d.getField("id"); + assertNotNull(id); + String ids = id.stringValue(); + assertFalse("99".equals(ids)); + StorableField f = d.getField("f"); + assertNotNull(f); + assertEquals(ids + " " + ids, f.stringValue()); + } + // check split outputs + for (Directory d : liveDocsMap.keySet()) { + BitSet b = liveDocsMap.get(d); + DirectoryReader r = DirectoryReader.open(d); + assertEquals(NUM_DOCS - 1, r.numDocs()); + assertEquals(NUM_DOCS - 1, r.maxDoc()); + assertEquals(0, r.numDeletedDocs()); + //System.err.println("----" + d); + for (int i = 0; i < r.maxDoc(); i++) { + StoredDocument doc = r.document(i); + StoredDocument mergeDoc = mergedReader.document(i); + if (b.get(i)) { + StorableField id = doc.getField("id"); + assertNotNull(id); + String ids = id.stringValue(); + assertFalse("99".equals(ids)); + StorableField f = doc.getField("f"); + assertNotNull(f); + assertEquals(ids + " " + ids, f.stringValue()); + StorableField mid = mergeDoc.getField("id"); + assertEquals(mid.stringValue(), id.stringValue()); + } else { + assertEquals("should be empty", 0, doc.getFields().size()); + } + } + // check terms + Fields fields = MultiFields.getFields(r); + Terms terms = fields.terms("id"); + assertNotNull("should have id", terms); + // XXX hmm, this sometimes fails. why? + //assertEquals(NUM_DOCS - 1, terms.getDocCount()); + TermsEnum te = terms.iterator(null); + BytesRef term = null; + int cnt = 0; + BitSet check = new BitSet(); + while ((term = te.next()) != null) { + cnt++; + int tid = Integer.parseInt(term.utf8ToString()); + check.set(tid); + } + // make sure all ids match the bits in bitsets + assertEquals(b, check); + // make sure we got the same number of ids as selected by bitset + assertEquals(b.cardinality(), cnt); + terms = fields.terms("f"); + assertNotNull("should have terms in field f", terms); + te = terms.iterator(null); + cnt = 0; + check.clear(); + while ((term = te.next()) != null) { + cnt++; + String[] vals = term.utf8ToString().split(" "); + int tid = Integer.parseInt(vals[0]); + check.set(tid); + } + // make sure field values match the bits in bitsets + assertEquals(b, check); + // make sure we got the same number of ids as selected by bitset + assertEquals(b.cardinality(), cnt); + r.close(); + } + mergedReader.close(); + merged.close(); + } +} Property changes on: lucene/misc/src/test/org/apache/lucene/index/TestTeeIndexSplitter.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/misc/src/java/org/apache/lucene/index/TeeIndexSplitter.java =================================================================== --- lucene/misc/src/java/org/apache/lucene/index/TeeIndexSplitter.java (revision 0) +++ lucene/misc/src/java/org/apache/lucene/index/TeeIndexSplitter.java (working copy) @@ -0,0 +1,247 @@ +package org.apache.lucene.index; +/** + * 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.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.analysis.core.WhitespaceAnalyzer; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.codecs.filtering.WriteFilter; +import org.apache.lucene.codecs.tee.TeeCodec; +import org.apache.lucene.codecs.tee.TeeDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.Version; + +/** + * This index splitter uses a combination of {@link TeeCodec} and {@link FilteringCodec} + * to push index data to multiple destinations in parallel and filter them on the fly. + * This approach allows doing a single pass over the input data, which should be faster + * than the {@link MultiPassIndexSplitter}. As a side-effect of this approach also a + * single merged index is created on output. + */ +public class TeeIndexSplitter { + + /** + * Split index into even parts. Parts will be saved to <dstDir>/part-NNNN + * directories, and a merged output will be saved to <dstDir>/merged + */ + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("Usage: TeeIndexSplitter "); + return; + } + Directory mainDir = FSDirectory.open(new File(args[0])); + DirectoryReader reader = DirectoryReader.open(mainDir); + File dstDir = new File(args[1]); + if (!dstDir.mkdirs()) { + System.err.println("ERROR: can't create output directory " + dstDir.getAbsolutePath()); + return; + } + File merged = new File(dstDir, "merged"); + merged.mkdirs(); + Directory mergedDir = FSDirectory.open(merged); + int parts = Integer.parseInt(args[2]); + Map liveDocsMap = new HashMap(); + int maxDoc = reader.maxDoc(); + int delta = maxDoc / parts; + if (delta < 1) delta = 1; + for (int i = 0; i < parts; i++) { + File out = new File(dstDir, "part-" + i); + out.mkdirs(); + Directory d = FSDirectory.open(out); + BitSet b = new BitSet(maxDoc); + for (int j = i * delta; j < (i + 1) * delta; j++) { + b.set(j); + } + liveDocsMap.put(d, b); + } + split(reader, mergedDir, liveDocsMap); + reader.close(); + mergedDir.close(); + for (Directory d : liveDocsMap.keySet()) { + d.close(); + } + } + + static class SplitterWriteFilter extends WriteFilter { + BitSet b; + + SplitterWriteFilter(BitSet b) { + this.b = b; + } + + @Override + public boolean filterStartDoc(PostingsConsumer original, FieldInfo info, + BytesRef text, int docID, int termDocFreq) throws IOException { + if (b.get(docID)) { + super.filterStartDoc(original, info, text, docID, termDocFreq); + return true; + } else { + return false; + } + } + + @Override + public void filterFinishDoc(PostingsConsumer original, FieldInfo info, + BytesRef text, int docID) throws IOException { + if (b.get(docID)) { + super.filterFinishDoc(original, info, text, docID); + } + } + + @Override + public void filterAddPosition(PostingsConsumer original, FieldInfo info, + BytesRef text, int docID, int position, BytesRef payload, + int startOffset, int endOffset) throws IOException { + if (b.get(docID)) { + super.filterAddPosition(original, info, text, docID, position, payload, + startOffset, endOffset); + } + } + + @Override + public boolean addDocValue(int docID, StorableField value, boolean norms) { + return b.get(docID); + } + + @Override + public List filterStoredFields(int docId, + List original) { + if (b.get(docId)) { + return original; + } else { + return null; + } + } + } + + /** + * Split index into parts defined by bitsets. All bitsets should be of size + * reader.maxDoc(), and they don't have to be disjoint (in which + * case data for a given document will end up in multiple destinations). + * @param r input index reader + * @param merged output directory for the merged index + * @param liveDocsMap map of output directories for parts and the bitsets + */ + public static void split(IndexReader r, Directory merged, Map liveDocsMap) throws Exception { + AtomicReader reader; + if (r instanceof AtomicReader) { + reader = (AtomicReader)r; + } else if (r instanceof CompositeReader) { + reader = new SlowCompositeReaderWrapper((CompositeReader)r); + } else { + throw new Exception("Unsupported reader type " + r.getClass().getName()); + } + split(reader, merged, liveDocsMap, null, false); + } + + /** + * Split index into parts defined by bitsets. All bitsets should be of size + * reader.maxDoc(), and they don't have to be disjoint (in which + * case data for a given document will end up in multiple destinations). + * @param reader input index reader + * @param merged destination directory for the merged index + * @param liveDocsMap map of output directories for parts and the bitsets + * @param codecsMap map of output directories and codecs to use for encoding data + * in that destination. Can be null, empty or partially filled - any missing + * codecs will be replaced with {@link Codec#getDefault()}. + * @param undeleteFirst if true then all documents in the input index are considered + * live (no deleted documents). + */ + public static void split(AtomicReader reader, Directory merged, + Map liveDocsMap, Map codecsMap, + boolean undeleteFirst) throws Exception { + Bits liveDocs = reader.getLiveDocs(); + if (liveDocs != null) { + if (!undeleteFirst) { + // roll-in current deletes into the liveDocsMap + for (int i = 0; i < liveDocs.length(); i++) { + if (!liveDocs.get(i)) { + for (BitSet b : liveDocsMap.values()) { + b.clear(i); + } + } + } + } else { + reader = new FilterAtomicReader(reader) { + @Override + public Bits getLiveDocs() { + return null; + } + }; + } + } + BitSet mainBits = new BitSet(reader.maxDoc()); + for (BitSet b : liveDocsMap.values()) { + mainBits.or(b); + } + // XXX HACK !!! + Directory wrappedDir = new TeeDirectory(merged, liveDocsMap.keySet(), + new HashSet(Arrays.asList( + IndexFileNames.SEGMENTS_GEN, + "_nrm.cfs", "_nrm.cfe", // compound norms + ".len" // SimpleText norms + )), false); + + if (codecsMap == null) { + codecsMap = Collections.emptyMap(); + } + HashMap outputs = new HashMap(); + for (Directory d : liveDocsMap.keySet()) { + Codec c = codecsMap.get(d); + if (c == null) { + c = Codec.getDefault(); + } + FilteringCodec fc = new FilteringCodec(c, new SplitterWriteFilter(liveDocsMap.get(d))); + outputs.put(d, fc); + } + + // XXX this doesn't work ... why? + // FilteringCodec mainFilter = new FilteringCodec(Codec.getDefault(), new SplitterWriteFilter(mainBits)); + // TeeCodec tee = new TeeCodec(mainFilter, outputs); + + TeeCodec tee = new TeeCodec(Codec.getDefault(), outputs); + IndexWriterConfig cfg = new IndexWriterConfig(Version.LUCENE_50, new WhitespaceAnalyzer(Version.LUCENE_50)); + cfg.setCodec(tee); + cfg.setMergeScheduler(new SerialMergeScheduler()); + MergePolicy mp = cfg.getMergePolicy(); + if (mp instanceof LogMergePolicy) { + ((LogMergePolicy)mp).setUseCompoundFile(false); + } else if (mp instanceof TieredMergePolicy) { + ((TieredMergePolicy)mp).setUseCompoundFile(false); + } else { + throw new Exception("Unsupported merge policy: " + mp.getClass().getName()); + } + IndexWriter writer = new IndexWriter(wrappedDir, cfg); + writer.addIndexes(reader); + writer.close(); + } + +} Property changes on: lucene/misc/src/java/org/apache/lucene/index/TeeIndexSplitter.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java (revision 1390869) +++ 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 1390869) +++ 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/tee/TestTeeCodec.java =================================================================== --- lucene/codecs/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java (revision 0) +++ lucene/codecs/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java (working copy) @@ -0,0 +1,353 @@ +package org.apache.lucene.codecs.tee; + +/* + * 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.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.analysis.core.WhitespaceAnalyzer; +import org.apache.lucene.codecs.Codec; +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.IndexFileNames; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableFieldType; +import org.apache.lucene.index.LogMergePolicy; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.SerialMergeScheduler; +import org.apache.lucene.index.StorableField; +import org.apache.lucene.index.StoredDocument; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.TieredMergePolicy; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RAMDirectory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.junit.Test; + +public class TestTeeCodec extends LuceneTestCase { + + Map outputs = new HashMap(); + List dirs = new ArrayList(); + Directory mainDir; + Codec mainCodec; + TeeCodec tee; + IndexWriterConfig cfg; + + @Override + public void setUp() throws Exception { + super.setUp(); + outputs.clear(); + dirs.clear(); + mainCodec = Codec.getDefault(); + boolean preFlex = mainCodec.getName().equals("Lucene3x"); + assumeFalse("TeeCodec doesn't work with Lucene3x codec", preFlex); + mainDir = new RAMDirectory(); + for (int i = 0; i < 10; i++) { + Directory d = new RAMDirectory(); + dirs.add(d); + outputs.put(d, mainCodec); + } + tee = new TeeCodec(mainCodec, outputs); + cfg = new IndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer( + TEST_VERSION_CURRENT)); + cfg.setCodec(tee); + cfg.setMergeScheduler(new SerialMergeScheduler()); + MergePolicy mp = cfg.getMergePolicy(); + if (mp instanceof LogMergePolicy) { + ((LogMergePolicy) mp).setUseCompoundFile(false); + } else if (mp instanceof TieredMergePolicy) { + ((TieredMergePolicy) mp).setUseCompoundFile(false); + } else { + fail("Unsupported merge policy: " + mp.getClass().getName()); + } + } + + @Test + public void testCodec() throws Exception { + Directory wrappedDir = new TeeDirectory(mainDir, dirs, + new HashSet(Arrays.asList( + IndexFileNames.SEGMENTS_GEN, + "_nrm.cfs", "_nrm.cfe", // compound norms + ".len" // SimpleText norms + )), false); + IndexWriter w = new IndexWriter(wrappedDir, cfg); + Document doc = new Document(); + FieldType ft = new FieldType(); + ft.setStored(true); + ft.setIndexed(true); + ft.setTokenized(true); + ft.setStoreTermVectors(true); + ft.setStoreTermVectorOffsets(true); + ft.setStoreTermVectorPositions(true); + doc.add(new Field("f1", "one two three", ft)); + doc.add(new Field("f2", "two three four five", ft)); + w.addDocument(doc); + doc = new Document(); + doc.add(new Field("f1", "six seven eight", ft)); + doc.add(new Field("f2", "eight nine ten", ft)); + doc.add(new Field("f3", "eleven twelve", ft)); + w.addDocument(doc); + w.close(); + for (Directory d : dirs) { +// for (String s : d.listAll()) { +// System.out.println(s + "\t" + d.fileLength(s)); +// } + DirectoryReader r = DirectoryReader.open(d); + // check doc 0 + StoredDocument storedDoc = r.document(0); + assertEquals("one two three", storedDoc.get("f1")); + assertEquals("two three four five", storedDoc.get("f2")); + // check doc 1 + storedDoc = r.document(1); + assertEquals("six seven eight", storedDoc.get("f1")); + assertEquals("eight nine ten", storedDoc.get("f2")); + assertEquals("eleven twelve", storedDoc.get("f3")); + // check terms + Terms terms = MultiFields.getTerms(r, "f1"); + assertEquals(6, terms.size()); + TermsEnum te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("one"), true)); + assertTrue(te.seekExact(new BytesRef("two"), true)); + assertTrue(te.seekExact(new BytesRef("three"), true)); + assertTrue(te.seekExact(new BytesRef("six"), true)); + assertTrue(te.seekExact(new BytesRef("seven"), true)); + assertTrue(te.seekExact(new BytesRef("eight"), true)); + terms = MultiFields.getTerms(r, "f2"); + assertEquals(7, terms.size()); + te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("two"), true)); + assertTrue(te.seekExact(new BytesRef("three"), true)); + assertTrue(te.seekExact(new BytesRef("four"), true)); + assertTrue(te.seekExact(new BytesRef("five"), true)); + assertTrue(te.seekExact(new BytesRef("eight"), true)); + assertTrue(te.seekExact(new BytesRef("nine"), true)); + assertTrue(te.seekExact(new BytesRef("ten"), true)); + terms = MultiFields.getTerms(r, "f3"); + assertEquals(2, terms.size()); + te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("eleven"), true)); + assertTrue(te.seekExact(new BytesRef("twelve"), true)); + r.close(); + } + } + + @Test + public void testMerging() throws Exception { + int CNT = 1280; + cfg.setMaxBufferedDocs(10); + MergePolicy mp = cfg.getMergePolicy(); + if (mp instanceof LogMergePolicy) { + ((LogMergePolicy)mp).setMergeFactor(2); + } + Directory wrappedDir = new TeeDirectory(mainDir, dirs, + new HashSet(Arrays.asList( + IndexFileNames.SEGMENTS_GEN, + "_nrm.cfs", "_nrm.cfe", // compound norms + ".len" // SimpleText norms + )), false); + IndexWriter w = new IndexWriter(wrappedDir, cfg); + FieldType ft = new FieldType(); + ft.setStored(true); + ft.setIndexed(true); + ft.setTokenized(true); + ft.setStoreTermVectors(true); + ft.setStoreTermVectorOffsets(true); + ft.setStoreTermVectorPositions(true); + for (int i = 0; i < CNT; i++) { + Document doc = new Document(); + doc.add(new Field("f1", "one " + i, ft)); + w.addDocument(doc); + } + w.close(); + List readers = new ArrayList(dirs.size()); + for (Directory d : dirs) { + DirectoryReader r = DirectoryReader.open(d); + assertEquals("should be " + CNT + " docs", CNT, r.maxDoc()); + assertEquals("should be no deletes", 0, r.numDeletedDocs()); + readers.add(r); + // check terms + Terms terms = MultiFields.getTerms(r, "f1"); + TermsEnum te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("one"), true)); + assertEquals(CNT, te.docFreq()); + for (int i = 0; i < CNT; i++) { + assertTrue(te.seekExact(new BytesRef("" + i), true)); + assertEquals(1, te.docFreq()); + } + } + for (int i = 0; i < CNT; i++) { + // check stored fields + StoredDocument doc = readers.get(0).document(i); + for (int j = 1; j < readers.size(); j++) { + StoredDocument newDoc = readers.get(j).document(i); + assertDocEquals(doc, newDoc); + } + } + } + + private void assertDocEquals(StoredDocument one, StoredDocument two) throws Exception { + assertNotNull("doc one", one); + assertNotNull("doc two", two); + List fieldsOne = one.getFields(); + List fieldsTwo = two.getFields(); + assertEquals("should have the same number of fields", fieldsOne.size(), fieldsTwo.size()); + for (StorableField f1 : fieldsOne) { + StorableField f2 = two.getField(f1.name()); + assertNotNull("should have field " + f1.name(), f2); + IndexableFieldType t1 = f1.fieldType(); + IndexableFieldType t2 = f2.fieldType(); + assertTrue("field types differ", + t1.indexed() == t2.indexed() && + t1.omitNorms() == t2.omitNorms() && + t1.stored() == t2.stored() && + t1.storeTermVectorOffsets() == t2.storeTermVectorOffsets() && + t1.storeTermVectorPositions() == t2.storeTermVectorPositions() && + t1.storeTermVectors() == t2.storeTermVectors() && + t1.tokenized() == t2.tokenized() && + t1.docValueType() == t2.docValueType() && + t1.indexOptions() == t2.indexOptions()); + assertEquals("binary values differ", f1.binaryValue(), f2.binaryValue()); + assertEquals("string values differ", f1.stringValue(), f2.stringValue()); + assertEquals("numeric values differ", f1.numericValue(), f2.numericValue()); + } + } + + @Test + public void testTeeDirectory() throws Exception { + Directory wrappedDir = new TeeDirectory(mainDir, dirs, null, false); + cfg.setCodec(Codec.getDefault()); + IndexWriter w = new IndexWriter(wrappedDir, cfg); + FieldType ft = new FieldType(); + ft.setStored(true); + ft.setIndexed(true); + ft.setTokenized(true); + ft.setStoreTermVectors(true); + ft.setStoreTermVectorOffsets(true); + ft.setStoreTermVectorPositions(true); + int CNT = 100; + for (int i = 0; i < CNT; i++) { + Document doc = new Document(); + doc.add(new Field("f1", "one " + i, ft)); + w.addDocument(doc); + } + w.close(); + List readers = new ArrayList(dirs.size()); + for (Directory d : dirs) { + DirectoryReader r = DirectoryReader.open(d); + assertEquals("should be " + CNT + " docs", CNT, r.maxDoc()); + assertEquals("should be no deletes", 0, r.numDeletedDocs()); + readers.add(r); + // check terms + Terms terms = MultiFields.getTerms(r, "f1"); + TermsEnum te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("one"), true)); + assertEquals(CNT, te.docFreq()); + for (int i = 0; i < CNT; i++) { + assertTrue(te.seekExact(new BytesRef("" + i), true)); + assertEquals(1, te.docFreq()); + } + } + for (int i = 0; i < CNT; i++) { + // check stored fields + StoredDocument doc = readers.get(0).document(i); + for (int j = 1; j < readers.size(); j++) { + StoredDocument newDoc = readers.get(j).document(i); + assertDocEquals(doc, newDoc); + } + } + } + + @Test + public void testInitialSync() throws Exception { + _testSync(false); + _testSync(true); + } + + private void cleanDir(Directory d) throws IOException { + String[] names = d.listAll(); + for (String name : names) { + d.deleteFile(name); + } + } + + private void _testSync(boolean initialSync) throws Exception { + // clean directories + cleanDir(mainDir); + for (Directory d : dirs) { + cleanDir(d); + } + // create initial files in the mainDir + for (int i = 0; i < 10; i++) { + IndexOutput out = mainDir.createOutput("initial" + i, IOContext.DEFAULT); + for (int j = 0; j < 10; j++) { + out.writeInt(j); + } + out.close(); + } + // now create a tee + Directory wrappedDir = new TeeDirectory(mainDir, dirs, null, initialSync); + // write some other junk + for (int i = 0; i < 10; i++) { + IndexOutput out = wrappedDir.createOutput("tee" + i, IOContext.DEFAULT); + for (int j = 0; j < 10; j++) { + out.writeInt(j); + } + out.close(); + } + for (Directory d : dirs) { + // make sure the initial files are absent/present depending on initialSync + for (int i = 0; i < 10; i++) { + if (initialSync) { + assertTrue("initial files should be mirrored", d.fileExists("initial" + i)); + // make sure they have the right content + IndexInput in = d.openInput("initial" + i, IOContext.READ); + for (int j = 0; j < 10; j++) { + int v = in.readInt(); + assertEquals("values read from file should be the same", v, j); + } + in.close(); + } else { + assertFalse("initial files should not be mirrored", d.fileExists("initial" + i)); + } + assertTrue("tee files should be there", d.fileExists("tee" + i)); + // make sure they have the right content + IndexInput in = d.openInput("tee" + i, IOContext.READ); + for (int j = 0; j < 10; j++) { + int v = in.readInt(); + assertEquals("values read from file should be the same", v, j); + } + in.close(); + } + } + } + +} Property changes on: lucene/codecs/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property 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 TestWriteFilter extends WriteFilter { + + TestWriteFilter() {} + + @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 TestWriteFilter()); + 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/tee/TeeDirectory.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java (working copy) @@ -0,0 +1,201 @@ +package org.apache.lucene.codecs.tee; + +/** + * 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 java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.lucene.codecs.tee.impl.TeeIndexOutput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.IOUtils; + +/** + * This class provides a 1:1 mirroring mechanism at the level of + * {@link Directory}. Unlike the {@link TeeCodec} it doesn't allow you to change + * the encoding of data, instead it simply copies byte by byte all data written + * to the main directory. + *

+ * You can specify a list of suffixes for file names to be mirrored. Files with + * names that don't match the list will be exempt from the mirroring process. If + * the list is empty or null then all files will be mirrored. + *

+ * The main Directory is used for all read operations. + *

+ * Note: existing data in the Directory won't be mirrored unless the + * initialSync is set to true - then in the constructor all + * existing data (from files with matching names) in the main directory will be + * copied, regardless whether files with the same names already exist in the + * secondary directories. See also the note above about name matching. + */ +public class TeeDirectory extends Directory { + + private Set suffixes; + private Collection dirs; + private Directory main; + private boolean debug = false; + + /** + * Create a TeeDirectory. + * @param main main directory + * @param dirs secondary directories. + * @param suffixes a list of file name suffixes that are subject to the mirroring. + * See the note above. + * @param initialSync if true then the data from already existing files (that + * match suffixes) will be copied to all secondary directories. If false then only newly + * created files will be copied. + * @throws IOException if an exception occurs during initial sync. + */ + public TeeDirectory(Directory main, Collection dirs, + Set suffixes, boolean initialSync) throws IOException { + this.main = main; + this.lockFactory = main.getLockFactory(); + this.dirs = dirs; + if (suffixes == null || suffixes.isEmpty()) { + this.suffixes = Collections.emptySet(); + } else { + this.suffixes = suffixes; + } + if (initialSync) { + initialSync(); + } + } + + private void initialSync() throws IOException { + String[] files = main.listAll(); + for (String name : files) { + if (!suffixMatch(name)) { + continue; + } + // copy the file + Map outs = new HashMap(); + for (Directory d : dirs) { + outs.put(d, d.createOutput(name, IOContext.DEFAULT)); + } + TeeIndexOutput tee = null; + IndexInput in = null; + try { + tee = new TeeIndexOutput(outs); + in = main.openInput(name, IOContext.READ); + tee.copyBytes(in, main.fileLength(name)); + } finally { + IOUtils.close(tee, in); + } + } + + } + + private boolean suffixMatch(String name) { + if (suffixes.isEmpty()) { + return true; + } + for (String s : suffixes) { + if (name.endsWith(s)) { + return true; + } + } + return false; + } + + @Override + public String[] listAll() throws IOException { + return main.listAll(); + } + + @Override + public boolean fileExists(String name) throws IOException { + return main.fileExists(name); + } + + @Override + public synchronized void deleteFile(String name) throws IOException { + main.deleteFile(name); + if (suffixMatch(name)) { + for (Directory d : dirs) { + d.deleteFile(name); + } + } + } + + @Override + public long fileLength(String name) throws IOException { + return main.fileLength(name); + } + + @Override + public synchronized IndexOutput createOutput(String name, IOContext context) + throws IOException { + if (debug) { + new Exception("-createOutput " + name).printStackTrace(); + } + IndexOutput mainOutput = main.createOutput(name, context); + if (suffixMatch(name)) { + Map outs = new HashMap(); + outs.put(main, mainOutput); + for (Directory d : dirs) { + outs.put(d, d.createOutput(name, context)); + } + return new TeeIndexOutput(outs); + } else { + return mainOutput; + } + } + + @Override + public synchronized void sync(Collection names) throws IOException { + main.sync(names); + if (suffixes.isEmpty()) { + for (Directory d : dirs) { + d.sync(names); + } + } else { + Set syncNames = new HashSet(); + for (String n : names) { + if (suffixMatch(n)) { + syncNames.add(n); + } + } + if (!syncNames.isEmpty()) { + for (Directory d : dirs) { + d.sync(syncNames); + } + } + } + } + + @Override + public IndexInput openInput(String name, IOContext context) + throws IOException { + return main.openInput(name, context); + } + + @Override + public synchronized void close() throws IOException { + main.close(); + for (Directory d : dirs) { + d.close(); + } + } +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/TeeCodec.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/TeeCodec.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/TeeCodec.java (working copy) @@ -0,0 +1,118 @@ +package org.apache.lucene.codecs.tee; + +/** + * 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.util.Map; + +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.tee.impl.TeeDocValuesFormat; +import org.apache.lucene.codecs.tee.impl.TeeFieldInfosFormat; +import org.apache.lucene.codecs.tee.impl.TeeLiveDocsFormat; +import org.apache.lucene.codecs.tee.impl.TeeNormsFormat; +import org.apache.lucene.codecs.tee.impl.TeePostingsFormat; +import org.apache.lucene.codecs.tee.impl.TeeSegmentInfosFormat; +import org.apache.lucene.codecs.tee.impl.TeeStoredFieldsFormat; +import org.apache.lucene.codecs.tee.impl.TeeTermVectorsFormat; +import org.apache.lucene.store.Directory; + +/** + * This codec implements a tee mechanism, that is it mirrors all index data + * written to it to multiple outputs, possible using different codec per output. + * + * Note: this codec is primarily intended as an output mechanism. The reading + * part uses just the main codec ignoring other codecs, and may not reflect + * correctly the state of data in all outputs. + */ +public class TeeCodec extends Codec { + + private final PostingsFormat postingsFormat; + private final DocValuesFormat docValuesFormat; + private final FieldInfosFormat fieldInfosFormat; + private final LiveDocsFormat liveDocsFormat; + private final NormsFormat normsFormat; + private final SegmentInfoFormat segmentInfoFormat; + private final StoredFieldsFormat storedFieldsFormat; + private final TermVectorsFormat termVectorsFormat; + + /** + * Create a TeeCodec. + * @param mainCodec main codec used for the managing the index. + * @param outputs a map of outputs where the data written to the main index + * will be mirrored, possibly using different codecs. + */ + public TeeCodec(Codec mainCodec, Map outputs) { + super(mainCodec.getName()); + postingsFormat = new TeePostingsFormat(mainCodec, outputs); + docValuesFormat = new TeeDocValuesFormat(mainCodec, outputs); + fieldInfosFormat = new TeeFieldInfosFormat(mainCodec, outputs); + liveDocsFormat = new TeeLiveDocsFormat(mainCodec, outputs); + normsFormat = new TeeNormsFormat(mainCodec, outputs); + segmentInfoFormat = new TeeSegmentInfosFormat(mainCodec, outputs); + storedFieldsFormat = new TeeStoredFieldsFormat(mainCodec, outputs); + termVectorsFormat = new TeeTermVectorsFormat(mainCodec, outputs); + } + + @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/tee/TeeCodec.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java (working copy) @@ -0,0 +1,49 @@ +package org.apache.lucene.codecs.tee.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.util.Map; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.SegmentInfoFormat; +import org.apache.lucene.codecs.SegmentInfoReader; +import org.apache.lucene.codecs.SegmentInfoWriter; +import org.apache.lucene.store.Directory; + +public class TeeSegmentInfosFormat extends SegmentInfoFormat { + Codec main; + Map outputs; + SegmentInfoFormat mainFormat; + + public TeeSegmentInfosFormat(Codec main, Map outputs) { + this.main = main; + this.outputs = outputs; + mainFormat = main.segmentInfoFormat(); + } + + @Override + public synchronized SegmentInfoReader getSegmentInfoReader() { + return mainFormat.getSegmentInfoReader(); + } + + @Override + public synchronized SegmentInfoWriter getSegmentInfoWriter() { + SegmentInfoWriter mainWriter = mainFormat.getSegmentInfoWriter(); + return new TeeSegmentInfoWriter(mainWriter, outputs); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java (working copy) @@ -0,0 +1,54 @@ +package org.apache.lucene.codecs.tee.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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FieldInfosWriter; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +public class TeeFieldInfosWriter extends FieldInfosWriter { + FieldInfosWriter main; + Map writers; + + public TeeFieldInfosWriter(FieldInfosFormat mainFormat, + Map outputs) throws IOException { + main = mainFormat.getFieldInfosWriter(); + writers = new HashMap(); + for (Entry e : outputs.entrySet()) { + writers.put(e.getKey(), e.getValue().fieldInfosFormat().getFieldInfosWriter()); + } + } + + @Override + public void write(Directory directory, String segmentName, FieldInfos infos, + IOContext context) throws IOException { + main.write(directory, segmentName, infos, context); + // write to each output directory + for (Entry e : writers.entrySet()) { + e.getValue().write(e.getKey(), segmentName, infos, context); + } + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java (working copy) @@ -0,0 +1,62 @@ +package org.apache.lucene.codecs.tee.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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.Directory; + +public class TeePostingsFormat extends PostingsFormat { + + private final PostingsFormat mainFormat; + private final Map outputs; + + public TeePostingsFormat(Codec main, Map outputs) { + super(main.getName()); + this.outputs = outputs; + this.mainFormat = main.postingsFormat(); + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + FieldsConsumer mainConsumer = mainFormat.fieldsConsumer(state); + HashMap consumers = new HashMap(); + for (Entry e : outputs.entrySet()) { + SegmentWriteState s = new SegmentWriteState(state.infoStream, e.getKey(), + state.segmentInfo, state.fieldInfos, state.termIndexInterval, + state.segDeletes, state.context); + consumers.put(e.getKey(), e.getValue().postingsFormat().fieldsConsumer(s)); + } + return new TeeFieldsConsumer(mainConsumer, consumers); + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { + return mainFormat.fieldsProducer(state); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java (working copy) @@ -0,0 +1,76 @@ +package org.apache.lucene.codecs.tee.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 java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.LiveDocsFormat; +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 TeeLiveDocsFormat extends LiveDocsFormat { + LiveDocsFormat mainFormat; + Map formats; + + public TeeLiveDocsFormat(Codec main, Map outputs) { + this.mainFormat = main.liveDocsFormat(); + this.formats = new HashMap(); + for (Entry e : outputs.entrySet()) { + formats.put(e.getKey(), e.getValue().liveDocsFormat()); + } + } + + @Override + public MutableBits newLiveDocs(int size) throws IOException { + return mainFormat.newLiveDocs(size); + } + + @Override + public MutableBits newLiveDocs(Bits existing) throws IOException { + return mainFormat.newLiveDocs(existing); + } + + @Override + public Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) + throws IOException { + return mainFormat.readLiveDocs(dir, info, context); + } + + @Override + public synchronized void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, + int newDelCount, IOContext context) throws IOException { + mainFormat.writeLiveDocs(bits, dir, info, newDelCount, context); + // write to each outputs directory + for (Entry e : formats.entrySet()) { + e.getValue().writeLiveDocs(bits, e.getKey(), info, newDelCount, context); + } + } + + @Override + public void files(SegmentInfoPerCommit info, Collection files) throws IOException { + mainFormat.files(info, files); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java (working copy) @@ -0,0 +1,58 @@ +package org.apache.lucene.codecs.tee.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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.IOUtils; + +public class TeeFieldsConsumer extends FieldsConsumer { + FieldsConsumer mainConsumer; + Map consumers; + + public TeeFieldsConsumer(FieldsConsumer mainConsumer, HashMap consumers) { + this.mainConsumer = mainConsumer; + this.consumers = consumers; + } + + @Override + public synchronized TermsConsumer addField(FieldInfo field) throws IOException { + TermsConsumer mainTConsumer = mainConsumer.addField(field); + HashMap tConsumers = new HashMap(); + for (Entry e : consumers.entrySet()) { + tConsumers.put(e.getKey(), e.getValue().addField(field)); + } + return new TeeTermsConsumer(mainTConsumer, tConsumers); + } + + @Override + public synchronized void close() throws IOException { + try { + mainConsumer.close(); + } finally { + IOUtils.close(consumers.values()); + } + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java (working copy) @@ -0,0 +1,61 @@ +package org.apache.lucene.codecs.tee.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.Map; + +import org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; + +public class TeePostingsConsumer extends PostingsConsumer { + PostingsConsumer mainConsumer; + Map consumers; + + public TeePostingsConsumer(PostingsConsumer mainConsumer, + Map consumers) { + this.mainConsumer = mainConsumer; + this.consumers = consumers; + } + + @Override + public synchronized void startDoc(int docID, int termDocFreq) throws IOException { + mainConsumer.startDoc(docID, termDocFreq); + for (PostingsConsumer c : consumers.values()) { + c.startDoc(docID, termDocFreq); + } + } + + @Override + public synchronized void addPosition(int position, BytesRef payload, int startOffset, + int endOffset) throws IOException { + mainConsumer.addPosition(position, payload, startOffset, endOffset); + for (PostingsConsumer c : consumers.values()) { + c.addPosition(position, payload, startOffset, endOffset); + } + } + + @Override + public synchronized void finishDoc() throws IOException { + mainConsumer.finishDoc(); + for (PostingsConsumer c : consumers.values()) { + c.finishDoc(); + } + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java (working copy) @@ -0,0 +1,81 @@ +package org.apache.lucene.codecs.tee.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.Map; + +import org.apache.lucene.codecs.StoredFieldsWriter; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.StorableField; +import org.apache.lucene.store.Directory; + +public class TeeStoredFieldsWriter extends StoredFieldsWriter { + + private final StoredFieldsWriter mainWriter; + private final Map writers; + + public TeeStoredFieldsWriter(StoredFieldsWriter mainWriter, + Map writers) { + this.mainWriter = mainWriter; + this.writers = writers; + } + + @Override + public void close() throws IOException { + mainWriter.close(); + for (StoredFieldsWriter w : writers.values()) { + w.close(); + } + } + + @Override + public void startDocument(int numStoredFields) throws IOException { + mainWriter.startDocument(numStoredFields); + for (StoredFieldsWriter w : writers.values()) { + w.startDocument(numStoredFields); + } + } + + @Override + public void writeField(FieldInfo info, StorableField field) + throws IOException { + mainWriter.writeField(info, field); + for (StoredFieldsWriter w : writers.values()) { + w.writeField(info, field); + } + } + + @Override + public void abort() { + mainWriter.abort(); + for (StoredFieldsWriter w : writers.values()) { + w.abort(); + } + } + + @Override + public void finish(FieldInfos fis, int numDocs) throws IOException { + mainWriter.finish(fis, numDocs); + for (StoredFieldsWriter w : writers.values()) { + w.finish(fis, numDocs); + } + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java (working copy) @@ -0,0 +1,62 @@ +package org.apache.lucene.codecs.tee.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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +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 TeeTermVectorsFormat extends TermVectorsFormat { + + private final TermVectorsFormat mainFormat; + private final Map outputs; + + public TeeTermVectorsFormat(Codec main, Map outputs) { + this.outputs = outputs; + mainFormat = main.termVectorsFormat(); + } + + @Override + public TermVectorsReader vectorsReader(Directory directory, + SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) + throws IOException { + return mainFormat.vectorsReader(directory, segmentInfo, fieldInfos, context); + } + + @Override + public synchronized TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo info, + IOContext context) throws IOException { + TermVectorsWriter mainWriter = mainFormat.vectorsWriter(directory, info, context); + Map writers = new HashMap(); + for (Entry e : outputs.entrySet()) { + writers.put(e.getKey(), e.getValue().termVectorsFormat().vectorsWriter(e.getKey(), info, context)); + } + return new TeeTermVectorsWriter(mainWriter, writers); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java (working copy) @@ -0,0 +1,140 @@ +package org.apache.lucene.codecs.tee.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.Iterator; +import java.util.Map; + +import org.apache.lucene.codecs.SegmentInfoWriter; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IndexOutput; + +/** + * Copy the main output to many secondary outputs. + */ +public class TeeIndexOutput extends IndexOutput { + + SegmentInfoWriter main; + Map outs; + Map writers; + + /** Create an output that writes to multiple destinations. */ + public TeeIndexOutput(Map outs) { + this(null, outs, null); + } + + /** + * Create an output that writes to multiple destinations. This + * constructor is used in the context of two-phase commit for + * segment infos. + * @param main main writer + * @param outs outputs + * @param writers writers that manage these outputs + */ + TeeIndexOutput(SegmentInfoWriter main, Map outs, + Map writers) { + this.main = main; + this.outs = outs; + this.writers = writers; + } + + /** + * Return the outputs. + */ + Map getOutputs() { + return outs; + } + + /** + * Return the writers. + */ + Map getWriters() { + return writers; + } + + /** + * Return the main writer. + */ + SegmentInfoWriter getMain() { + return main; + } + + @Override + public synchronized void setLength(long length) throws IOException { + for (IndexOutput io : outs.values()) { + io.setLength(length); + } + } + + @Override + public synchronized void flush() throws IOException { + for (IndexOutput io : outs.values()) { + io.flush(); + } + } + + @Override + public synchronized void close() throws IOException { + for (IndexOutput io : outs.values()) { + io.close(); + } + } + + @Override + public long getFilePointer() { + // should be the same + Iterator it = outs.values().iterator(); + long ptr = it.next().getFilePointer(); + if (outs.size() > 1) { + assert it.next().getFilePointer() == ptr; + } + return ptr; + } + + @Override + public synchronized void seek(long pos) throws IOException { + for (IndexOutput io : outs.values()) { + io.seek(pos); + } + } + + @Override + public long length() throws IOException { + // should be the same + Iterator it = outs.values().iterator(); + long len = it.next().length(); + if (outs.size() > 1) { + assert it.next().length() == len; + } + return len; + } + + @Override + public synchronized void writeByte(byte b) throws IOException { + for (IndexOutput io : outs.values()) { + io.writeByte(b); + } + } + + @Override + public synchronized void writeBytes(byte[] b, int offset, int length) throws IOException { + for (IndexOutput io : outs.values()) { + io.writeBytes(b, offset, length); + } + } +} \ No newline at end of file Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java (working copy) @@ -0,0 +1,71 @@ +package org.apache.lucene.codecs.tee.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 java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; + +public class TeeTermsConsumer extends TermsConsumer { + TermsConsumer main; + Map consumers; + + public TeeTermsConsumer(TermsConsumer main, Map consumers) { + this.main = main; + this.consumers = consumers; + } + + @Override + public synchronized PostingsConsumer startTerm(BytesRef text) throws IOException { + PostingsConsumer mainPConsumer = main.startTerm(text); + HashMap pConsumers = new HashMap(); + for (Entry e : consumers.entrySet()) { + pConsumers.put(e.getKey(), e.getValue().startTerm(text)); + } + return new TeePostingsConsumer(mainPConsumer, pConsumers); + } + + @Override + public synchronized void finishTerm(BytesRef text, TermStats stats) throws IOException { + main.finishTerm(text, stats); + for (TermsConsumer tc : consumers.values()) { + tc.finishTerm(text, stats); + } + } + + @Override + public synchronized void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) + throws IOException { + main.finish(sumTotalTermFreq, sumDocFreq, docCount); + for (TermsConsumer tc : consumers.values()) { + tc.finish(sumTotalTermFreq, sumDocFreq, docCount); + } + } + + @Override + public Comparator getComparator() throws IOException { + return main.getComparator(); + } +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java (working copy) @@ -0,0 +1,66 @@ +package org.apache.lucene.codecs.tee.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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.codecs.PerDocProducer; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.store.Directory; + +public class TeeNormsFormat extends NormsFormat { + + private final NormsFormat mainFormat; + private final Map outputs; + + public TeeNormsFormat(Codec main, Map outputs) { + this.outputs = outputs; + mainFormat = main.normsFormat(); + } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + PerDocConsumer mainConsumer = mainFormat.docsConsumer(state); + // create states for each output + HashMap consumers = new HashMap(); + for (Entry e : outputs.entrySet()) { + PerDocWriteState s = new PerDocWriteState(state.infoStream, + e.getKey(), state.segmentInfo, state.bytesUsed, + state.segmentSuffix, state.context); + //System.err.println(state.segmentName + "/" + state.segmentSuffix); + PerDocConsumer c = e.getValue().docValuesFormat().docsConsumer(s); + if (c != null) { + consumers.put(e.getKey(), c); + } + } + return new TeePerDocConsumer(mainConsumer, consumers); + } + + @Override + public PerDocProducer docsProducer(SegmentReadState state) throws IOException { + return mainFormat.docsProducer(state); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java (working copy) @@ -0,0 +1,105 @@ +package org.apache.lucene.codecs.tee.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 java.util.Map; + +import org.apache.lucene.codecs.TermVectorsWriter; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; + +public class TeeTermVectorsWriter extends TermVectorsWriter { + + private final TermVectorsWriter mainWriter; + private final Map writers; + + public TeeTermVectorsWriter(TermVectorsWriter mainWriter, + Map writers) { + this.mainWriter = mainWriter; + this.writers = writers; + } + + @Override + public void close() throws IOException { + mainWriter.close(); + for (TermVectorsWriter w : writers.values()) { + w.close(); + } + } + + @Override + public void startDocument(int numVectorFields) throws IOException { + mainWriter.startDocument(numVectorFields); + for (TermVectorsWriter w : writers.values()) { + w.startDocument(numVectorFields); + } + } + + + @Override + public void startField(FieldInfo info, int numTerms, boolean positions, + boolean offsets, boolean payloads) throws IOException { + mainWriter.startField(info, numTerms, positions, offsets, payloads); + for (TermVectorsWriter w : writers.values()) { + w.startField(info, numTerms, positions, offsets, payloads); + } + } + + @Override + public void startTerm(BytesRef term, int freq) throws IOException { + mainWriter.startTerm(term, freq); + for (TermVectorsWriter w : writers.values()) { + w.startTerm(term, freq); + } + } + + @Override + public void addPosition(int position, int startOffset, int endOffset, BytesRef payload) + throws IOException { + mainWriter.addPosition(position, startOffset, endOffset, payload); + for (TermVectorsWriter w : writers.values()) { + w.addPosition(position, startOffset, endOffset, payload); + } + } + + @Override + public void abort() { + mainWriter.abort(); + for (TermVectorsWriter w : writers.values()) { + w.abort(); + } + } + + @Override + public synchronized void finish(FieldInfos fis, int numDocs) throws IOException { + mainWriter.finish(fis, numDocs); + for (TermVectorsWriter w : writers.values()) { + w.finish(fis, numDocs); + } + } + + @Override + public Comparator getComparator() throws IOException { + return mainWriter.getComparator(); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfoWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfoWriter.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfoWriter.java (working copy) @@ -0,0 +1,50 @@ +package org.apache.lucene.codecs.tee.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.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +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 TeeSegmentInfoWriter extends SegmentInfoWriter { + + private final SegmentInfoWriter mainWriter; + private final Map outputs; + + public TeeSegmentInfoWriter(SegmentInfoWriter mainWriter, Map outputs) { + this.mainWriter = mainWriter; + this.outputs = outputs; + } + + @Override + public void write(Directory dir, SegmentInfo info, FieldInfos fis, + IOContext ioContext) throws IOException { + mainWriter.write(dir, info, fis, ioContext); + for (Entry e : outputs.entrySet()) { + e.getValue().segmentInfoFormat().getSegmentInfoWriter().write(e.getKey(), info, fis, ioContext); + } + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfoWriter.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java (working copy) @@ -0,0 +1,105 @@ +package org.apache.lucene.codecs.tee.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.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.index.AtomicReader; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.DocValues.Type; +import org.apache.lucene.store.Directory; + +public class TeePerDocConsumer extends PerDocConsumer { + PerDocConsumer mainConsumer; + Map consumers; + + public TeePerDocConsumer(PerDocConsumer mainConsumer, + Map consumers) throws IOException { + this.mainConsumer = mainConsumer; + this.consumers = consumers; + } + + @Override + public synchronized void close() throws IOException { + if (mainConsumer != null) { + mainConsumer.close(); + } + for (Entry e : consumers.entrySet()) { + PerDocConsumer c = e.getValue(); + if (c != null) { + c.close(); + } + } + } + + @Override + public synchronized DocValuesConsumer addValuesField(Type type, FieldInfo field) + throws IOException { + return new TeeDocValuesConsumer(mainConsumer, consumers, type, field); + } + + @Override + public synchronized void abort() { + if (mainConsumer != null) { + mainConsumer.abort(); + } + for (Entry e : consumers.entrySet()) { + PerDocConsumer c = e.getValue(); + if (c != null) { + c.abort(); + } + } + } + + // HACKY: this is trappy about perdocconsumer API + // must delete since all norms impl this differently... + // we should restructure + @Override + public boolean canMerge(FieldInfo info) { + if (mainConsumer != null) { + return mainConsumer.canMerge(info); + } else { + return false; + } + } + + @Override + public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException { + if (mainConsumer != null) { + return mainConsumer.getDocValuesForMerge(reader, info); + } else { + return null; + } + } + + @Override + public Type getDocValuesType(FieldInfo info) { + if (mainConsumer != null) { + return mainConsumer.getDocValuesType(info); + } else { + return null; + } + } + + + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java (working copy) @@ -0,0 +1,52 @@ +package org.apache.lucene.codecs.tee.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.Map; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FieldInfosReader; +import org.apache.lucene.codecs.FieldInfosWriter; +import org.apache.lucene.store.Directory; + +public class TeeFieldInfosFormat extends FieldInfosFormat { + + private final FieldInfosFormat mainFormat; + private final Map outputs; + + public TeeFieldInfosFormat(Codec main, Map outputs) { + this.outputs = outputs; + mainFormat = main.fieldInfosFormat(); + } + + /** + * Read always the original output. + */ + @Override + public FieldInfosReader getFieldInfosReader() throws IOException { + return mainFormat.getFieldInfosReader(); + } + + @Override + public synchronized FieldInfosWriter getFieldInfosWriter() throws IOException { + return new TeeFieldInfosWriter(mainFormat, outputs); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java (working copy) @@ -0,0 +1,69 @@ +package org.apache.lucene.codecs.tee.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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.codecs.PerDocProducer; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.store.Directory; + +public class TeeDocValuesFormat extends DocValuesFormat { + + private final DocValuesFormat mainFormat; + private final Map outputs; + + public TeeDocValuesFormat(Codec main, Map outputs) { + this.outputs = outputs; + mainFormat = main.docValuesFormat(); + } + + @Override + public synchronized PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + PerDocConsumer mainConsumer = mainFormat.docsConsumer(state); + // create states for each output + HashMap consumers = new HashMap(); + for (Entry e : outputs.entrySet()) { + PerDocWriteState s = new PerDocWriteState(state.infoStream, + e.getKey(), state.segmentInfo, state.bytesUsed, + state.segmentSuffix, state.context); + PerDocConsumer c = e.getValue().docValuesFormat().docsConsumer(s); + // consumer may be null if output codec doesn't support it + if (c != null) { + consumers.put(e.getKey(), c); + } + } + return new TeePerDocConsumer(mainConsumer, consumers); + } + + /** + * Reads always from the main codec. + */ + @Override + public PerDocProducer docsProducer(SegmentReadState state) throws IOException { + return mainFormat.docsProducer(state); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java (working copy) @@ -0,0 +1,61 @@ +package org.apache.lucene.codecs.tee.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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.StoredFieldsReader; +import org.apache.lucene.codecs.StoredFieldsWriter; +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 TeeStoredFieldsFormat extends StoredFieldsFormat { + + private final StoredFieldsFormat mainFormat; + private final Map outputs; + + public TeeStoredFieldsFormat(Codec main, Map outputs) { + this.outputs = outputs; + mainFormat = main.storedFieldsFormat(); + } + + @Override + public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, + FieldInfos fn, IOContext context) throws IOException { + return mainFormat.fieldsReader(directory, si, fn, context); + } + + @Override + public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, + IOContext context) throws IOException { + StoredFieldsWriter mainWriter = mainFormat.fieldsWriter(directory, si, context); + HashMap writers = new HashMap(); + for (Entry e : outputs.entrySet()) { + writers.put(e.getKey(), e.getValue().storedFieldsFormat().fieldsWriter(e.getKey(), si, context)); + } + return new TeeStoredFieldsWriter(mainWriter, writers); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java (working copy) @@ -0,0 +1,75 @@ +package org.apache.lucene.codecs.tee.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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.index.DocValues.Type; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.StorableField; +import org.apache.lucene.store.Directory; + +public class TeeDocValuesConsumer extends DocValuesConsumer { + + private final Type type; + private final DocValuesConsumer mainDVConsumer; + private final Map dvConsumers; + + public TeeDocValuesConsumer(PerDocConsumer mainConsumer, + Map consumers, Type type, FieldInfo field) + throws IOException { + this.type = type; + mainDVConsumer = mainConsumer.addValuesField(type, field); + dvConsumers = new HashMap(); + for (Entry e : consumers.entrySet()) { + dvConsumers.put(e.getKey(), e.getValue().addValuesField(type, field)); + } + } + + @Override + public void add(int docID, StorableField value) throws IOException { + mainDVConsumer.add(docID, value); + for (DocValuesConsumer c : dvConsumers.values()) { + c.add(docID, value); + } + } + + @Override + public void finish(int docCount) throws IOException { + mainDVConsumer.finish(docCount); + for (DocValuesConsumer c : dvConsumers.values()) { + c.finish(docCount); + } + } + + @Override + protected Type getType() { + return type; + } + + @Override + public int getValueSize() { + return mainDVConsumer.getValueSize(); + } + +} Property changes on: lucene/codecs/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.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/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,250 @@ +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/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,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.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,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.TermVectorsFormat; +import org.apache.lucene.codecs.TermVectorsReader; +import org.apache.lucene.codecs.TermVectorsWriter; +import org.apache.lucene.codecs.filtering.FilteringCodec; +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(FilteringCodec filtering) { + this.original = filtering.originalCodec.termVectorsFormat(); + } + + @Override + public TermVectorsReader vectorsReader(Directory directory, + SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) + throws IOException { + TermVectorsReader reader = original.vectorsReader(directory, segmentInfo, fieldInfos, context); + return reader; + } + + @Override + public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo info, + IOContext context) throws IOException { + TermVectorsWriter writer = original.vectorsWriter(directory, info, context); + return writer; + } + +} 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,46 @@ +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; +import org.apache.lucene.codecs.filtering.FilteringCodec; + +public class FilteringSegmentInfoFormat extends SegmentInfoFormat { + + private final FilteringCodec filtering; + private final SegmentInfoFormat original; + + public FilteringSegmentInfoFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.segmentInfoFormat(); + } + + @Override + public SegmentInfoReader getSegmentInfoReader() { + return original.getSegmentInfoReader(); + } + + @Override + public SegmentInfoWriter getSegmentInfoWriter() { + SegmentInfoWriter writer = original.getSegmentInfoWriter(); + return new FilteringSegmentInfoWriter(filtering, writer); + } + +} 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,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.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 { + + final WriteFilter filter; + final PerDocConsumer original; + + private final boolean norms; + + FilteringPerDocConsumer(WriteFilter filter, PerDocConsumer original, boolean norms) { + this.filter = filter; + 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(this, type, field, norms); + } 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.FilteringCodec; + +public class FilteringFieldInfosFormat extends FieldInfosFormat { + + final FilteringCodec filtering; + final FieldInfosFormat original; + + public FilteringFieldInfosFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.fieldInfosFormat(); + } + + @Override + public FieldInfosReader getFieldInfosReader() throws IOException { + return original.getFieldInfosReader(); + } + + @Override + public FieldInfosWriter getFieldInfosWriter() throws IOException { + if (filtering.writeFilter == null) { + return original.getFieldInfosWriter(); + } else { + return new FilteringFieldInfosWriter(this); + } + } + +} 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/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,58 @@ +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.FilteringCodec; +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 FilteringCodec filtering; + private final StoredFieldsFormat original; + + public FilteringStoredFieldsFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.storedFieldsFormat(); + } + + @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 (filtering.writeFilter == null) { + return writer; + } else { + return new FilteringStoredFieldsWriter(filtering.writeFilter, writer); + } + } + +} 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,91 @@ +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 filter; + private final TermsConsumer original; + private final FieldInfo info; + + private BytesRef lastStartTerm = null; + private FilteringPostingsConsumer fpc; + + public FilteringTermsConsumer(WriteFilter filter, FieldInfo info, TermsConsumer original) throws IOException { + this.filter = 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 (!filter.startTerm(info, text)) { + return NoopPostingsConsumer.INSTANCE; + } + fpc = new FilteringPostingsConsumer(filter, original, info, text); + 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; + } + filter.filterFinishTerm(original, info, text, stats); + } + + @Override + public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) + throws IOException { + filter.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.FilteringCodec; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentReadState; + +public class FilteringNormsFormat extends NormsFormat { + + private final FilteringCodec filtering; + private final NormsFormat original; + + public FilteringNormsFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.normsFormat(); + } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + if (filtering.writeFilter == null) { + return original.docsConsumer(state); + } else { + PerDocConsumer consumer = original.docsConsumer(state); + return new FilteringPerDocConsumer(filtering.writeFilter, consumer, true); + } + } + + @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,27 @@ +package org.apache.lucene.codecs.filtering.impl; + +import java.io.IOException; + +import org.apache.lucene.codecs.SegmentInfoWriter; +import org.apache.lucene.codecs.filtering.FilteringCodec; +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(FilteringCodec filtering, + 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,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.FieldInfosWriter; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +class FilteringFieldInfosWriter extends FieldInfosWriter { + + private final FilteringFieldInfosFormat filtering; + private final FieldInfosWriter original; + + FilteringFieldInfosWriter(FilteringFieldInfosFormat filtering) throws IOException { + this.filtering = filtering; + this.original = filtering.original.getFieldInfosWriter(); + } + + @Override + public void write(Directory directory, String segmentName, FieldInfos infos, + IOContext context) throws IOException { + FieldInfos filteredInfos = filtering.filtering.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,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.FieldsConsumer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; + +public class FilteringPostingsFormat extends PostingsFormat { + + private final FilteringCodec filtering; + private final PostingsFormat original; + + public FilteringPostingsFormat(FilteringCodec filtering) { + super(filtering.originalCodec.getName()); + this.filtering = filtering; + this.original = filtering.originalCodec.postingsFormat(); + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) + throws IOException { + if (filtering.writeFilter == null) { + return original.fieldsConsumer(state); + } else { + FieldsConsumer consumer = original.fieldsConsumer(state); + return new FilteringFieldsConsumer(filtering.writeFilter, consumer); + } + } + + @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,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 java.util.Collection; + +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.filtering.FilteringCodec; +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 FilteringCodec filtering; + private final LiveDocsFormat original; + + public FilteringLiveDocsFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.liveDocsFormat(); + } + + @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 (filtering.writeFilter == null) { + original.writeLiveDocs(bits, dir, info, newDelCount, context); + } else { + MutableBits filteredBits = filtering.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,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.FieldsConsumer; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.codecs.filtering.WriteFilter; +import org.apache.lucene.index.FieldInfo; + +class FilteringFieldsConsumer extends FieldsConsumer { + WriteFilter filter; + FieldsConsumer original; + + FilteringFieldsConsumer(WriteFilter filter, FieldsConsumer original) { + this.filter = filter; + this.original = original; + } + + @Override + public TermsConsumer addField(FieldInfo field) throws IOException { + if (filter == null) { + return original.addField(field); + } else { + if (filter.addField(field)) { + TermsConsumer consumer = original.addField(field); + return new FilteringTermsConsumer(filter, field, consumer); + } 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,75 @@ +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 filter; + 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(WriteFilter filter, TermsConsumer originalTermsConsumer, + FieldInfo info, BytesRef text) throws IOException { + this.filter = filter; + 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 (filter.filterStartDoc(original, info, text, docID, termDocFreq)) { + numDocsStarted++; + totalTermFreq += termDocFreq; + } + } + + @Override + public void addPosition(int position, BytesRef payload, int startOffset, + int endOffset) throws IOException { + + filter.filterAddPosition(original, info, text, curDocID, position, payload, + startOffset, endOffset); + } + + @Override + public void finishDoc() throws IOException { + filter.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 { + + WriteFilter filter; + StoredFieldsWriter original; + List fields = null; + int docNum; + int numDocs; + + public FilteringStoredFieldsWriter(WriteFilter filter, StoredFieldsWriter original) { + this.filter = filter; + 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 = filter.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,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 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.FilteringCodec; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentReadState; + +public class FilteringDocValuesFormat extends DocValuesFormat { + + private final FilteringCodec filtering; + private final DocValuesFormat original; + + public FilteringDocValuesFormat(FilteringCodec filtering) { + this.filtering = filtering; + original = filtering.originalCodec.docValuesFormat(); + } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + if (filtering.writeFilter == null) { + return original.docsConsumer(state); + } else { + PerDocConsumer consumer = original.docsConsumer(state); + return new FilteringPerDocConsumer(filtering.writeFilter, consumer, false); + } + } + + @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.index.DocValues.Type; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.StorableField; + +public class FilteringDocValuesConsumer extends DocValuesConsumer { + + private final FilteringPerDocConsumer filtering; + private final DocValuesConsumer original; + private final boolean norms; + private final Type type; + + FilteringDocValuesConsumer(FilteringPerDocConsumer filtering, + Type type, FieldInfo field, boolean norms) throws IOException { + this.filtering = filtering; + this.original = filtering.original.addValuesField(type, field); + this.norms = norms; + this.type = type; + } + + @Override + public void add(int docID, StorableField value) throws IOException { + if (filtering.filter.addDocValue(docID, value, norms)) { + StorableField val = filtering.filter.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/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,119 @@ +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 { + + public final Codec originalCodec; + public final WriteFilter writeFilter; + + 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 originalCodec original codec used to encode the data. + * @param writeFilter subclass of {@link WriteFilter}, or null for no + * filtering. + */ + public FilteringCodec(Codec originalCodec, WriteFilter writeFilter) { + super(originalCodec.getName()); + this.originalCodec = originalCodec; + this.writeFilter = writeFilter; + postingsFormat = new FilteringPostingsFormat(this); + docValuesFormat = new FilteringDocValuesFormat(this); + storedFieldsFormat = new FilteringStoredFieldsFormat(this); + termVectorsFormat = new FilteringTermVectorsFormat(this); + fieldInfosFormat = new FilteringFieldInfosFormat(this); + segmentInfoFormat = new FilteringSegmentInfoFormat(this); + normsFormat = new FilteringNormsFormat(this); + liveDocsFormat = new FilteringLiveDocsFormat(this); + } + + @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 Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java (revision 1390869) +++ 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(); }