Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java	(revision 1244135)
+++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java	(working copy)
@@ -84,19 +84,22 @@
       super(state, segmentSuffix);
     }
 
+    // nocommit
     @Override
-    protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
+    public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
         throws IOException {
       return reader.normValues(info.name);
     }
 
+    // nocommit
     @Override
-    protected boolean canMerge(FieldInfo info) {
+    public boolean canMerge(FieldInfo info) {
       return info.normsPresent();
     }
 
+    // nocommit
     @Override
-    protected Type getDocValuesType(FieldInfo info) {
+    public Type getDocValuesType(FieldInfo info) {
       return info.getNormType();
     }
     
Index: lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java	(revision 1244135)
+++ lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java	(working copy)
@@ -71,7 +71,8 @@
    * the default override this method accordingly.
    * <p>
    */
-  protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException {
+  //nocommit
+  public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException {
     return reader.docValues(info.name);
   }
   
@@ -83,7 +84,8 @@
    * the default override this method accordingly.
    * <p>
    */
-  protected boolean canMerge(FieldInfo info) {
+  //nocommit
+  public boolean canMerge(FieldInfo info) {
     return info.hasDocValues();
   }
   
@@ -95,7 +97,8 @@
    * the default override this method accordingly.
    * <p>
    */
-  protected Type getDocValuesType(FieldInfo info) {
+  //nocommit
+  public Type getDocValuesType(FieldInfo info) {
     return info.getDocValuesType();
   }
   
Index: lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java
===================================================================
--- lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java	(revision 1244135)
+++ lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java	(working copy)
@@ -86,19 +86,22 @@
     }
   }
   
+  // nocommit
   @Override
-  protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
+  public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
       throws IOException {
     return reader.normValues(info.name);
   }
 
+  // nocommit
   @Override
-  protected boolean canMerge(FieldInfo info) {
+  public boolean canMerge(FieldInfo info) {
     return info.normsPresent();
   }
 
+  // nocommit
   @Override
-  protected Type getDocValuesType(FieldInfo info) {
+  public Type getDocValuesType(FieldInfo info) {
     return info.getNormType();
   }
 
Index: lucene/contrib/misc/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java
===================================================================
--- lucene/contrib/misc/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java	(revision 0)
+++ lucene/contrib/misc/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java	(working copy)
@@ -0,0 +1,119 @@
+package org.apache.lucene.codecs.filtering;
+
+import java.io.IOException;
+
+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.FieldsEnum;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.MultiFields;
+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;
+import org.apache.lucene.util.Version;
+
+public class TestFilteringCodec extends LuceneTestCase {
+  
+  BytesRef oneTerm = new BytesRef("one");
+  
+  private class TestWriteFilter extends WriteFilter {
+
+    @Override
+    public boolean addField(FieldInfo field) {
+      if (field.name.equals("w1")) {
+        return false;
+      }
+      return true;
+    }
+
+    @Override
+    public boolean startTerm(FieldInfo info, BytesRef text) {
+      if (info.name.equals("w2") && text.equals(oneTerm)) {
+        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(oneTerm)) {
+        return;
+      }
+      original.finishTerm(text, stats);
+    }
+
+    @Override
+    public boolean writeField(FieldInfo info, IndexableField field) {
+      if (info.name.equals("w1")) {
+        return false;
+      } else {
+        return true;
+      }
+    }    
+  }
+  
+  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(Version.LUCENE_40,
+        new WhitespaceAnalyzer(Version.LUCENE_40));
+    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());
+    doc = r.document(0);
+    assertNull("should 'throw away'", doc.get("w1"));
+    assertEquals("one two three", doc.get("w2"));
+    Fields fields = MultiFields.getFields(r);
+    FieldsEnum fe = fields.iterator();
+    String field;
+    TermsEnum te = null;
+    while ((field = fe.next()) != null) {
+      //System.out.println("field: " + field);
+      if (!preFlex) {
+        assertFalse("w1".equals(field)); // should be filtered out completely
+      }
+      Terms terms = fe.terms();
+      te = terms.iterator(te);
+      BytesRef term;
+      int cnt = 0;
+      while ((term = te.next()) != null) {
+        cnt++;
+        if ("w2".equals(field)) {
+         assertFalse(term.equals(oneTerm));
+        }
+      }
+      if (preFlex && "w1".equals(field)) {
+        assertEquals(0, cnt); // should be no terms
+      }
+    }
+  }
+}

Property changes on: lucene/contrib/misc/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java
===================================================================
--- lucene/contrib/misc/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java	(revision 0)
+++ lucene/contrib/misc/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java	(working copy)
@@ -0,0 +1,229 @@
+package org.apache.lucene.codecs.tee;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+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.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.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+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.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.index.TieredMergePolicy;
+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.apache.lucene.util.Version;
+import org.junit.Test;
+
+public class TestTeeCodec extends LuceneTestCase {
+  Map<Directory,Codec> outputs = new HashMap<Directory,Codec>();
+  List<Directory> dirs = new ArrayList<Directory>();
+  Directory mainDir;
+  Codec mainCodec;
+  TeeCodec tee;
+  IndexWriterConfig cfg;
+  
+  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(Version.LUCENE_40,
+        new WhitespaceAnalyzer(Version.LUCENE_40));
+    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<String>(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
+      doc = r.document(0);
+      assertEquals("one two three", doc.get("f1"));
+      assertEquals("two three four five", doc.get("f2"));
+      // check doc 1
+      doc = r.document(1);
+      assertEquals("six seven eight", doc.get("f1"));
+      assertEquals("eight nine ten", doc.get("f2"));
+      assertEquals("eleven twelve", doc.get("f3"));
+      // check terms
+      Terms terms = MultiFields.getTerms(r, "f1");
+      assertEquals(6, terms.getUniqueTermCount());
+      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.getUniqueTermCount());
+      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.getUniqueTermCount());
+      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<String>(Arrays.asList(
+                    IndexFileNames.SEGMENTS_GEN,
+                    "_nrm.cfs", "_nrm.cfe", // compound norms
+                    ".len" // SimpleText norms
+                    )), true);
+    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++) {
+      System.out.println(i);
+      Document doc = new Document();
+      doc.add(new Field("f1", "one " + i, ft));
+      w.addDocument(doc);
+    }
+    w.close();
+    for (Directory d : dirs) {
+      DirectoryReader r = DirectoryReader.open(d);
+      assertEquals(CNT, r.numDocs());
+      // check stored fields
+      for (int i = 0; i < CNT; i++) {
+        Document doc = r.document(i);
+        assertEquals("one " + i, doc.get("f1"));
+      }
+      // 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());
+      }
+    }
+  }
+  
+  @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++) {
+      System.out.println(i);
+      Document doc = new Document();
+      doc.add(new Field("f1", "one " + i, ft));
+      w.addDocument(doc);
+    }
+    w.close();
+    for (Directory d : dirs) {
+      DirectoryReader r = DirectoryReader.open(d);
+      assertEquals(CNT, r.numDocs());
+      // check stored fields
+      for (int i = 0; i < CNT; i++) {
+        Document doc = r.document(i);
+        assertEquals("one " + i, doc.get("f1"));
+      }
+      // 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());
+      }
+    }
+  }
+}

Property changes on: lucene/contrib/misc/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java	(working copy)
@@ -0,0 +1,115 @@
+package org.apache.lucene.codecs.filtering;
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.FieldInfosWriter;
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.MutableBits;
+
+/**
+ * 
+ */
+public class WriteFilter {
+
+  // fields
+  public boolean addField(FieldInfo field) {
+    return true;
+  }
+
+  // terms
+  public boolean startTerm(FieldInfo info, BytesRef text) {
+    return true;
+  }
+
+  public void filterStartTerm(TermsConsumer original, FieldInfo info,
+          BytesRef text) throws IOException {
+    original.startTerm(text);
+  }
+
+  public void filterFinishTerm(TermsConsumer original, FieldInfo info,
+          BytesRef text, TermStats stats) throws IOException {
+    original.finishTerm(text, stats);
+  }
+  
+  public void filterFinishTerms(TermsConsumer original, FieldInfo info,
+          long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
+    original.finish(sumTotalTermFreq, sumDocFreq, docCount);
+  }
+
+  public void filterStartDoc(PostingsConsumer original, BytesRef text,
+          int docID, int termDocFreq) throws IOException {
+    original.startDoc(docID, termDocFreq);
+  }
+
+  public void filterFinishDoc(PostingsConsumer original, BytesRef text,
+          int curDocID) throws IOException {
+    original.finishDoc();
+  }
+
+  public void filterAddPosition(PostingsConsumer original, BytesRef text,
+          int curDocID, int position, BytesRef payload, int startOffset,
+          int endOffset) throws IOException {
+    original.addPosition(position, payload, startOffset, endOffset);
+  }
+
+  // -------- FieldInfos ----------
+  public FieldInfos filterFieldInfos(Directory directory, String segmentName,
+          FieldInfos infos, IOContext context) {
+    return infos;
+  }
+  
+  // -------- DocValues -----------
+  public boolean addValuesField(Type type, FieldInfo field) {
+    return true;
+  }
+
+  /**
+   * 
+   * @param docID
+   * @param value
+   * @param norms true if the value represents Lucene norms
+   * @return
+   */
+  public boolean addDocValue(int docID, IndexableField value, boolean norms) {
+    return true;
+  }
+
+  /**
+   * 
+   * @param docID
+   * @param value
+   * @param norms true if the value represents Lucene norms
+   * @return the original or modified value. Must never return null.
+   * @see #addDocValue(int, IndexableField)
+   */
+  public IndexableField filterDocValue(int docID, IndexableField value, boolean norms) {
+    return value;
+  }
+
+  // -------- LiveDocs ----------
+  public MutableBits filterLiveDocs(MutableBits bits, Directory dir,
+          SegmentInfo info, IOContext context) {
+    return bits;
+  }
+
+  // Norms
+  public boolean addNormsField(Type type, FieldInfo field) {
+    return true;
+  }
+
+  // StoredFieldsWriter
+  public boolean writeField(FieldInfo info, IndexableField field) {
+    return true;
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java	(working copy)
@@ -0,0 +1,40 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import java.io.IOException;
+import java.util.Set;
+
+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;
+import org.apache.lucene.index.SegmentInfo;
+
+public class FilteringFieldInfosFormat extends FieldInfosFormat {
+  FilteringCodec filtering;
+  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);
+    }
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    original.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java	(working copy)
@@ -0,0 +1,43 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import java.io.IOException;
+import java.util.Set;
+
+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.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+
+public class FilteringDocValuesFormat extends DocValuesFormat {
+  FilteringCodec filtering;
+  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);
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    original.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java	(working copy)
@@ -0,0 +1,46 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import java.io.IOException;
+import java.util.Set;
+
+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 {
+  FilteringCodec filtering;
+  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, String segment,
+          IOContext context) throws IOException {
+    StoredFieldsWriter writer = original.fieldsWriter(directory, segment, context);
+    if (filtering.writeFilter == null) {
+      return writer;
+    } else {
+      return new FilteringStoredFieldsWriter(filtering.writeFilter, writer);
+    }
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    original.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java	(working copy)
@@ -0,0 +1,37 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+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.IndexableField;
+
+public class FilteringDocValuesConsumer extends DocValuesConsumer {
+  FilteringPerDocConsumer filtering;
+  DocValuesConsumer original;
+  boolean norms;
+  int filteredDocCount = 0;
+  
+  FilteringDocValuesConsumer(FilteringPerDocConsumer filtering,
+          Type type, FieldInfo field, boolean norms) throws IOException {
+    this.filtering = filtering;
+    this.original = filtering.original.addValuesField(type, field);
+    this.norms = norms;
+  }
+
+  @Override
+  public void add(int docID, IndexableField value) throws IOException {
+    if (filtering.filter.addDocValue(docID, value, norms)) {
+      filteredDocCount = docID;
+      IndexableField val = filtering.filter.filterDocValue(docID, value, norms);
+      original.add(docID, val);
+    }
+  }
+
+  @Override
+  public void finish(int docCount) throws IOException {
+    original.finish(docCount);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java	(working copy)
@@ -0,0 +1,28 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+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/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfosFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfosFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfosFormat.java	(working copy)
@@ -0,0 +1,31 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import org.apache.lucene.codecs.SegmentInfosFormat;
+import org.apache.lucene.codecs.SegmentInfosReader;
+import org.apache.lucene.codecs.SegmentInfosWriter;
+import org.apache.lucene.codecs.filtering.FilteringCodec;
+
+public class FilteringSegmentInfosFormat extends SegmentInfosFormat {
+  FilteringCodec filtering;
+  SegmentInfosFormat original;
+  
+  public FilteringSegmentInfosFormat(FilteringCodec filtering) {
+    this.filtering = filtering;
+    this.original = filtering.originalCodec.segmentInfosFormat();
+  }
+  
+  // XXX
+  @Override
+  public SegmentInfosReader getSegmentInfosReader() {
+    SegmentInfosReader reader = original.getSegmentInfosReader();
+    return reader;
+  }
+
+  // XXX
+  @Override
+  public SegmentInfosWriter getSegmentInfosWriter() {
+    SegmentInfosWriter writer = original.getSegmentInfosWriter();
+    return writer;
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfosFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java	(working copy)
@@ -0,0 +1,27 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+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 {
+  FilteringFieldInfosFormat filtering;
+  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, context);
+    original.write(directory, segmentName, filteredInfos, context);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java	(working copy)
@@ -0,0 +1,47 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import java.io.IOException;
+import java.util.Set;
+
+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.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+public class FilteringPostingsFormat extends PostingsFormat {
+  FilteringCodec filtering;
+  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);
+  }
+
+  @Override
+  public void files(SegmentInfo segmentInfo, String segmentSuffix,
+          Set<String> files) throws IOException {
+    original.files(segmentInfo, segmentSuffix, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java	(working copy)
@@ -0,0 +1,56 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.filtering.FilteringCodec;
+import org.apache.lucene.index.SegmentInfo;
+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 {
+  FilteringCodec filtering;
+  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, SegmentInfo info, IOContext context)
+          throws IOException {
+    return original.readLiveDocs(dir, info, context);
+  }
+
+  @Override
+  public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info,
+          IOContext context) throws IOException {
+    if (filtering.writeFilter == null) {
+      original.writeLiveDocs(bits, dir, info, context);
+    } else {
+      MutableBits filteredBits = 
+              filtering.writeFilter.filterLiveDocs(bits, dir, info, context);
+      original.writeLiveDocs(filteredBits, dir, info, context);
+    }
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    original.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java	(working copy)
@@ -0,0 +1,38 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+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/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java	(working copy)
@@ -0,0 +1,41 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.filtering.WriteFilter;
+import org.apache.lucene.util.BytesRef;
+
+public class FilteringPostingsConsumer extends PostingsConsumer {
+  WriteFilter filter;
+  BytesRef text;
+  PostingsConsumer original;
+  int curDocID;
+  
+  public FilteringPostingsConsumer(WriteFilter filter, PostingsConsumer original,
+          BytesRef text) throws IOException {
+    this.filter = filter;
+    this.text = text;
+    this.original = original;
+  }
+
+  @Override
+  public void startDoc(int docID, int termDocFreq) throws IOException {
+    curDocID = docID;
+    filter.filterStartDoc(original, text, docID, termDocFreq);
+  }
+
+  @Override
+  public void addPosition(int position, BytesRef payload, int startOffset,
+          int endOffset) throws IOException {
+    
+    filter.filterAddPosition(original, text, curDocID, position, payload,
+            startOffset, endOffset);
+  }
+
+  @Override
+  public void finishDoc() throws IOException {
+    filter.filterFinishDoc(original, text, curDocID);
+  }
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java	(working copy)
@@ -0,0 +1,80 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+
+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.index.FieldInfo;
+import org.apache.lucene.index.IndexableField;
+
+public class FilteringStoredFieldsWriter extends StoredFieldsWriter {
+  WriteFilter filter;
+  StoredFieldsWriter original;
+  List<FieldInfo> infos = null;
+  List<IndexableField> fields = null;
+  
+  public FilteringStoredFieldsWriter(WriteFilter filter, StoredFieldsWriter original) {
+    this.filter = filter;
+    this.original = original;
+  }
+
+  @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<IndexableField>(numStoredFields);
+    infos = new ArrayList<FieldInfo>(numStoredFields);
+  }
+
+  @Override
+  public void writeField(FieldInfo info, IndexableField field)
+          throws IOException {
+    infos.add(info);
+    fields.add(field);
+  }
+
+  @Override
+  public void abort() {
+    original.abort();
+    infos = null;
+    fields = null;
+  }
+
+  private void flush() throws IOException {
+    ArrayList<FieldInfo> filteredInfos = new ArrayList<FieldInfo>(infos.size());
+    ArrayList<IndexableField> filteredFields = new ArrayList<IndexableField>(fields.size());
+    for (int i = 0; i < fields.size(); i++) {
+      if (filter.writeField(infos.get(i), fields.get(i))) {
+        filteredInfos.add(infos.get(i));
+        filteredFields.add(fields.get(i));
+      }
+    }
+    original.startDocument(filteredFields.size());
+    for (int i = 0; i < filteredFields.size(); i++) {
+      original.writeField(filteredInfos.get(i), filteredFields.get(i));
+    }
+    infos = null;
+    fields = null;
+  }
+  
+  @Override
+  public void finish(int numDocs) throws IOException {
+    if (fields != null) {
+      flush();
+    }
+    original.finish(numDocs);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java	(working copy)
@@ -0,0 +1,46 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import java.io.IOException;
+import java.util.Set;
+
+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 {
+  FilteringCodec filtering;
+  TermVectorsFormat original;
+
+  public FilteringTermVectorsFormat(FilteringCodec filtering) {
+    this.filtering = filtering;
+    this.original = filtering.originalCodec.termVectorsFormat();
+  }
+  
+  // XXX
+  @Override
+  public TermVectorsReader vectorsReader(Directory directory,
+          SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context)
+          throws IOException {
+    TermVectorsReader reader = original.vectorsReader(directory, segmentInfo, fieldInfos, context);
+    return reader;
+  }
+
+  // XXX
+  @Override
+  public TermVectorsWriter vectorsWriter(Directory directory, String segment,
+          IOContext context) throws IOException {
+    TermVectorsWriter writer = original.vectorsWriter(directory, segment, context);
+    return writer;
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    original.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java	(working copy)
@@ -0,0 +1,34 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+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<BytesRef> getComparator() throws IOException {
+    return BytesRef.getUTF8SortedAsUnicodeComparator();
+  }
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java	(working copy)
@@ -0,0 +1,50 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+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 {
+  WriteFilter filter;
+  TermsConsumer original;
+  FieldInfo info;
+  
+  public FilteringTermsConsumer(WriteFilter filter, FieldInfo info, TermsConsumer original) throws IOException {
+    this.filter = filter;
+    this.original = original;
+    this.info = info;
+  }
+
+  @Override
+  public PostingsConsumer startTerm(final BytesRef text) throws IOException {
+    if (!filter.startTerm(info, text)) {
+      return NoopPostingsConsumer.INSTANCE;
+    }
+    PostingsConsumer consumer = original.startTerm(text);
+    FilteringPostingsConsumer fpc = new FilteringPostingsConsumer(filter, consumer, text);
+    return fpc;
+  }
+
+  @Override
+  public void finishTerm(final BytesRef text, TermStats stats) throws IOException {
+    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<BytesRef> getComparator() throws IOException {
+    return original.getComparator();
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java	(working copy)
@@ -0,0 +1,43 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import java.io.IOException;
+import java.util.Set;
+
+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.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+
+public class FilteringNormsFormat extends NormsFormat {
+  FilteringCodec filtering;
+  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);
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    original.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java	(working copy)
@@ -0,0 +1,50 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+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.FieldInfo;
+import org.apache.lucene.index.DocValues.Type;
+
+class FilteringPerDocConsumer extends PerDocConsumer {
+  WriteFilter filter;
+  PerDocConsumer original;
+  boolean norms;
+  
+  FilteringPerDocConsumer(WriteFilter filter, PerDocConsumer original, boolean norms) {
+    this.filter = filter;
+    this.original = original;
+    this.norms = norms;
+  }
+
+  @Override
+  public void close() throws IOException {
+    original.close();
+  }
+
+  @Override
+  public DocValuesConsumer addValuesField(Type type, FieldInfo field)
+          throws IOException {
+    if (norms) {
+      if (filter.addNormsField(type, field)) {
+        return new FilteringDocValuesConsumer(this, type, field, true);        
+      } else {
+        return NoopDocValuesConsumer.INSTANCE;
+      }
+    } else {
+      if (filter.addValuesField(type, field)) {
+        return new FilteringDocValuesConsumer(this, type, field, false);
+      } else {
+        return NoopDocValuesConsumer.INSTANCE;
+      }
+    }
+  }
+
+  @Override
+  public void abort() {
+    original.abort();
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java	(working copy)
@@ -0,0 +1,21 @@
+package org.apache.lucene.codecs.filtering.impl;
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.IndexableField;
+
+class NoopDocValuesConsumer extends DocValuesConsumer {
+  static final DocValuesConsumer INSTANCE = new NoopDocValuesConsumer();
+
+  @Override
+  public void add(int docID, IndexableField value) throws IOException {
+    // no-op
+  }
+
+  @Override
+  public void finish(int docCount) throws IOException {
+    // no-op
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java	(working copy)
@@ -0,0 +1,94 @@
+package org.apache.lucene.codecs.filtering;
+
+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.SegmentInfosFormat;
+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.FilteringSegmentInfosFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringStoredFieldsFormat;
+import org.apache.lucene.codecs.filtering.impl.FilteringTermVectorsFormat;
+
+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 FilteringSegmentInfosFormat segmentInfosFormat;
+  private final FilteringNormsFormat normsFormat;
+  private final FilteringLiveDocsFormat liveDocsFormat;
+  
+  /**
+   * 
+   * @param originalCodec
+   * @param writeFilter
+   * @param readFilter
+   */
+  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);
+    segmentInfosFormat = new FilteringSegmentInfosFormat(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 SegmentInfosFormat segmentInfosFormat() {
+    return segmentInfosFormat;
+  }
+
+  @Override
+  public NormsFormat normsFormat() {
+    return normsFormat;
+  }
+
+  @Override
+  public LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java	(working copy)
@@ -0,0 +1,44 @@
+package org.apache.lucene.codecs.tee.impl;
+
+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.IndexableField;
+import org.apache.lucene.store.Directory;
+
+public class TeeDocValuesConsumer extends DocValuesConsumer {
+  DocValuesConsumer mainDVConsumer;
+  Map<Directory,DocValuesConsumer> dvConsumers;
+
+  public TeeDocValuesConsumer(PerDocConsumer mainConsumer,
+          Map<Directory, PerDocConsumer> consumers, Type type, FieldInfo field) throws IOException {
+    mainDVConsumer = mainConsumer.addValuesField(type, field);
+    dvConsumers = new HashMap<Directory, DocValuesConsumer>();
+    for (Entry<Directory,PerDocConsumer> e : consumers.entrySet()) {
+      dvConsumers.put(e.getKey(), e.getValue().addValuesField(type, field));
+    }
+  }
+
+  @Override
+  public synchronized void add(int docID, IndexableField value) throws IOException {
+    mainDVConsumer.add(docID, value);
+    for (DocValuesConsumer c : dvConsumers.values()) {
+      c.add(docID, value);
+    }
+  }
+
+  @Override
+  public synchronized void finish(int docCount) throws IOException {
+    mainDVConsumer.finish(docCount);
+    for (DocValuesConsumer c : dvConsumers.values()) {
+      c.finish(docCount);
+    }
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java	(working copy)
@@ -0,0 +1,35 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.SegmentInfosFormat;
+import org.apache.lucene.codecs.SegmentInfosReader;
+import org.apache.lucene.codecs.SegmentInfosWriter;
+import org.apache.lucene.store.Directory;
+
+public class TeeSegmentInfosFormat extends SegmentInfosFormat {
+  Codec main;
+  Map<Directory,Codec> outputs;
+  SegmentInfosFormat mainFormat;
+  
+  public TeeSegmentInfosFormat(Codec main, Map<Directory,Codec> outputs) {
+    this.main = main;
+    this.outputs = outputs;
+    mainFormat = main.segmentInfosFormat();
+  }
+
+  @Override
+  public synchronized SegmentInfosReader getSegmentInfosReader() {
+    return mainFormat.getSegmentInfosReader();
+  }
+
+  @Override
+  public synchronized SegmentInfosWriter getSegmentInfosWriter() {
+    SegmentInfosWriter mainWriter = mainFormat.getSegmentInfosWriter();
+    return new TeeSegmentInfosWriter(mainWriter, outputs);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java	(working copy)
@@ -0,0 +1,58 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+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.PerDocWriteState;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.Directory;
+
+public class TeePostingsFormat extends PostingsFormat {
+  Codec main;
+  PostingsFormat mainFormat;
+  Map<Directory,Codec> outputs;
+
+  public TeePostingsFormat(Codec main, Map<Directory,Codec> outputs) {
+    super(main.getName());
+    this.main = main;
+    this.outputs = outputs;
+    this.mainFormat = main.postingsFormat();
+  }
+
+
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
+          throws IOException {
+    FieldsConsumer mainConsumer = mainFormat.fieldsConsumer(state);
+    HashMap<Directory,FieldsConsumer> consumers = new HashMap<Directory,FieldsConsumer>();
+    for (Entry<Directory,Codec> e : outputs.entrySet()) {
+      SegmentWriteState s = new SegmentWriteState(state.infoStream, e.getKey(),
+              state.segmentName, state.fieldInfos, state.numDocs, state.termIndexInterval,
+              e.getValue(), 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);
+  }
+
+  @Override
+  public void files(SegmentInfo segmentInfo, String segmentSuffix,
+          Set<String> files) throws IOException {
+    mainFormat.files(segmentInfo, segmentSuffix, files);
+  }
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java	(working copy)
@@ -0,0 +1,38 @@
+package org.apache.lucene.codecs.tee.impl;
+
+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<Directory,FieldInfosWriter> writers;
+
+  public TeeFieldInfosWriter(FieldInfosFormat mainFormat,
+          Map<Directory, Codec> outputs) throws IOException {
+    main = mainFormat.getFieldInfosWriter();
+    writers = new HashMap<Directory, FieldInfosWriter>();
+    for (Entry<Directory,Codec> e : outputs.entrySet()) {
+      writers.put(e.getKey(), e.getValue().fieldInfosFormat().getFieldInfosWriter());
+    }
+  }
+
+  @Override
+  public synchronized 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<Directory,FieldInfosWriter> e : writers.entrySet()) {
+      e.getValue().write(e.getKey(), segmentName, infos, context);
+    }
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java	(working copy)
@@ -0,0 +1,60 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.index.SegmentInfo;
+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<Directory,LiveDocsFormat> formats;
+  
+  public TeeLiveDocsFormat(Codec main, Map<Directory,Codec> outputs) {
+    this.mainFormat = main.liveDocsFormat();
+    this.formats = new HashMap<Directory, LiveDocsFormat>();
+    for (Entry<Directory,Codec> 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, SegmentInfo info, IOContext context)
+          throws IOException {
+    return mainFormat.readLiveDocs(dir, info, context);
+  }
+
+  @Override
+  public synchronized void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info,
+          IOContext context) throws IOException {
+    mainFormat.writeLiveDocs(bits, dir, info, context);
+    // write to each outputs directory
+    for (Entry<Directory,LiveDocsFormat> e : formats.entrySet()) {
+      e.getValue().writeLiveDocs(bits, e.getKey(), info, context);
+    }
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    mainFormat.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java	(working copy)
@@ -0,0 +1,41 @@
+package org.apache.lucene.codecs.tee.impl;
+
+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;
+
+public class TeeFieldsConsumer extends FieldsConsumer {
+  FieldsConsumer mainConsumer;
+  Map<Directory,FieldsConsumer> consumers;
+
+  public TeeFieldsConsumer(FieldsConsumer mainConsumer,
+          HashMap<Directory, FieldsConsumer> consumers) {
+    this.mainConsumer = mainConsumer;
+    this.consumers = consumers;
+  }
+
+  @Override
+  public TermsConsumer addField(FieldInfo field) throws IOException {
+    TermsConsumer mainTConsumer = mainConsumer.addField(field);
+    HashMap<Directory,TermsConsumer> tConsumers = new HashMap<Directory, TermsConsumer>();
+    for (Entry<Directory,FieldsConsumer> e : consumers.entrySet()) {
+      tConsumers.put(e.getKey(), e.getValue().addField(field));
+    }
+    return new TeeTermsConsumer(mainTConsumer, tConsumers);
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    mainConsumer.close();
+    for (FieldsConsumer f : consumers.values()) {
+      f.close();
+    }
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java	(working copy)
@@ -0,0 +1,46 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+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<Directory,PostingsConsumer> consumers;
+
+  public TeePostingsConsumer(PostingsConsumer mainConsumer,
+          Map<Directory, PostingsConsumer> 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/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java	(working copy)
@@ -0,0 +1,63 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.store.Directory;
+
+public class TeeStoredFieldsWriter extends StoredFieldsWriter {
+  StoredFieldsWriter mainWriter;
+  Map<Directory,StoredFieldsWriter> writers;
+
+  public TeeStoredFieldsWriter(StoredFieldsWriter mainWriter,
+          Map<Directory, StoredFieldsWriter> writers) {
+    this.mainWriter = mainWriter;
+    this.writers = writers;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    mainWriter.close();
+    for (StoredFieldsWriter w : writers.values()) {
+      w.close();
+    }
+  }
+
+  @Override
+  public synchronized void startDocument(int numStoredFields) throws IOException {
+    mainWriter.startDocument(numStoredFields);
+    for (StoredFieldsWriter w : writers.values()) {
+      w.startDocument(numStoredFields);
+    }
+  }
+
+  @Override
+  public synchronized void writeField(FieldInfo info, IndexableField field)
+          throws IOException {
+    mainWriter.writeField(info, field);
+    for (StoredFieldsWriter w : writers.values()) {
+      w.writeField(info, field);
+    }
+  }
+
+  @Override
+  public synchronized void abort() {
+    mainWriter.abort();
+    for (StoredFieldsWriter w : writers.values()) {
+      w.abort();
+    }
+  }
+
+  @Override
+  public synchronized void finish(int numDocs) throws IOException {
+    mainWriter.finish(numDocs);
+    for (StoredFieldsWriter w : writers.values()) {
+      w.finish(numDocs);
+    }
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java	(working copy)
@@ -0,0 +1,52 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+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 {
+  Codec main;
+  TermVectorsFormat mainFormat;
+  Map<Directory,Codec> outputs;
+  
+  public TeeTermVectorsFormat(Codec main, Map<Directory,Codec> outputs) {
+    this.main = main;
+    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, String segment,
+          IOContext context) throws IOException {
+    TermVectorsWriter mainWriter = mainFormat.vectorsWriter(directory, segment, context);
+    Map<Directory,TermVectorsWriter> writers = new HashMap<Directory,TermVectorsWriter>();
+    for (Entry<Directory,Codec> e : outputs.entrySet()) {
+      writers.put(e.getKey(), e.getValue().termVectorsFormat().vectorsWriter(e.getKey(), segment, context));
+    }
+    return new TeeTermVectorsWriter(mainWriter, writers);
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    mainFormat.files(info, files);
+  }
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java	(working copy)
@@ -0,0 +1,111 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.lucene.codecs.SegmentInfosWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexOutput;
+
+public class TeeIndexOutput extends IndexOutput {
+
+  SegmentInfosWriter main;
+  Map<Directory,IndexOutput> outs;
+  Map<Directory,SegmentInfosWriter> writers;
+  
+  public TeeIndexOutput(Map<Directory,IndexOutput> outs) {
+    this(null, outs, null);
+  }
+  
+  public TeeIndexOutput(SegmentInfosWriter main, Map<Directory,IndexOutput> outs,
+          Map<Directory,SegmentInfosWriter> writers) {
+    this.main = main;
+    this.outs = outs;
+    this.writers = writers;
+  }
+  
+  Map<Directory,IndexOutput> getOutputs() {
+    return outs;
+  }
+  
+  Map<Directory,SegmentInfosWriter> getWriters() {
+    return writers;
+  }
+  
+  SegmentInfosWriter getMain() {
+    return main;
+  }
+  
+  @Override
+  public synchronized void setLength(long length) throws IOException {
+    //System.err.println("-setLength " + length);
+    for (IndexOutput io : outs.values()) {
+      io.setLength(length);
+    }
+  }
+
+  @Override
+  public synchronized void flush() throws IOException {
+    //System.err.println("-flush");
+    for (IndexOutput io : outs.values()) {
+      io.flush();
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    //System.err.println("-close");
+    for (IndexOutput io : outs.values()) {
+      io.close();
+    }
+  }
+
+  @Override
+  public long getFilePointer() {
+    //System.err.println("-getFilePointer");
+    // should be the same
+    Iterator<IndexOutput> 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 {
+    //System.err.println("-seek " + pos);
+    for (IndexOutput io : outs.values()) {
+      io.seek(pos);
+    }
+  }
+
+  @Override
+  public long length() throws IOException {
+    //System.err.println("-length");
+    // should be the same
+    Iterator<IndexOutput> 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 {
+    //System.err.println("-writeByte" + b);
+    for (IndexOutput io : outs.values()) {
+      io.writeByte(b);
+    }
+  }
+
+  @Override
+  public synchronized void writeBytes(byte[] b, int offset, int length) throws IOException {
+    //System.err.println("-writeBytes " + b.length + "/" + offset + "/" + length);
+    for (IndexOutput io : outs.values()) {
+      io.writeBytes(b, offset, length);
+    }
+  }
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosWriter.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosWriter.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosWriter.java	(working copy)
@@ -0,0 +1,63 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.SegmentInfosWriter;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+
+public class TeeSegmentInfosWriter extends SegmentInfosWriter {
+  SegmentInfosWriter mainWriter;
+  Map<Directory,Codec> outputs;
+
+  public TeeSegmentInfosWriter(SegmentInfosWriter mainWriter,
+          Map<Directory,Codec> outputs) {
+    this.mainWriter = mainWriter;
+    this.outputs = outputs;
+  }
+
+  @Override
+  public synchronized IndexOutput writeInfos(Directory dir, String segmentsFileName,
+          String codecID, SegmentInfos infos, IOContext context)
+          throws IOException {
+    Map<Directory,IndexOutput> outs = new HashMap<Directory,IndexOutput>(outputs.size() + 1);
+    outs.put(dir, mainWriter.writeInfos(dir, segmentsFileName, codecID, infos, context));
+    Map<Directory,SegmentInfosWriter> writers = new HashMap<Directory,SegmentInfosWriter>();
+    writers.put(dir, mainWriter);
+    for (Entry<Directory,Codec> e : outputs.entrySet()) {
+      SegmentInfosWriter w = e.getValue().segmentInfosFormat().getSegmentInfosWriter();
+      outs.put(e.getKey(), w.writeInfos(e.getKey(), segmentsFileName, e.getValue().getName(), infos, context));
+      writers.put(e.getKey(), w);
+    }
+    return new TeeIndexOutput(mainWriter, outs, writers);
+  }
+
+  @Override
+  public synchronized void prepareCommit(IndexOutput out) throws IOException {
+    assert out instanceof TeeIndexOutput;
+    Map<Directory,IndexOutput> outs = ((TeeIndexOutput)out).getOutputs();
+    Map<Directory,SegmentInfosWriter> writers = ((TeeIndexOutput)out).getWriters();
+    for (Entry<Directory,SegmentInfosWriter> e : writers.entrySet()) {
+      e.getValue().prepareCommit(outs.get(e.getKey()));
+    }
+  }
+
+  @Override
+  public synchronized void finishCommit(IndexOutput out) throws IOException {
+    assert out instanceof TeeIndexOutput;
+    Map<Directory,IndexOutput> outs = ((TeeIndexOutput)out).getOutputs();
+    Map<Directory,SegmentInfosWriter> writers = ((TeeIndexOutput)out).getWriters();
+    for (Entry<Directory,SegmentInfosWriter> e : writers.entrySet()) {
+      e.getValue().finishCommit(outs.get(e.getKey()));
+    }
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java	(working copy)
@@ -0,0 +1,56 @@
+package org.apache.lucene.codecs.tee.impl;
+
+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<Directory,TermsConsumer> consumers;
+
+  public TeeTermsConsumer(TermsConsumer main,
+          Map<Directory, TermsConsumer> consumers) {
+    this.main = main;
+    this.consumers = consumers;
+  }
+
+  @Override
+  public synchronized PostingsConsumer startTerm(BytesRef text) throws IOException {
+    PostingsConsumer mainPConsumer = main.startTerm(text);
+    HashMap<Directory,PostingsConsumer> pConsumers = new HashMap<Directory, PostingsConsumer>();
+    for (Entry<Directory,TermsConsumer> 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<BytesRef> getComparator() throws IOException {
+    return main.getComparator();
+  }
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java	(working copy)
@@ -0,0 +1,62 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.store.Directory;
+
+public class TeeNormsFormat extends NormsFormat {
+  Codec main;
+  NormsFormat mainFormat;
+  Map<Directory,Codec> outputs;
+  
+  public TeeNormsFormat(Codec main, Map<Directory,Codec> outputs) {
+    this.main = main;
+    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<Directory,PerDocConsumer> consumers = new HashMap<Directory, PerDocConsumer>();
+    for (Entry<Directory,Codec> e : outputs.entrySet()) {
+      PerDocWriteState s = new PerDocWriteState(state.infoStream,
+              e.getKey(), state.segmentName, state.fieldInfos, 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);
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    HashSet<String> dbg = new HashSet<String>();
+    mainFormat.files(info, dbg);
+    //System.err.println("-files " + dbg);
+    mainFormat.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java	(working copy)
@@ -0,0 +1,85 @@
+package org.apache.lucene.codecs.tee.impl;
+
+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.store.Directory;
+import org.apache.lucene.util.BytesRef;
+
+public class TeeTermVectorsWriter extends TermVectorsWriter {
+  TermVectorsWriter mainWriter;
+  Map<Directory,TermVectorsWriter> writers;
+
+  public TeeTermVectorsWriter(TermVectorsWriter mainWriter,
+          Map<Directory, TermVectorsWriter> writers) {
+    this.mainWriter = mainWriter;
+    this.writers = writers;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    mainWriter.close();
+    for (TermVectorsWriter w : writers.values()) {
+      w.close();
+    }
+  }
+
+  @Override
+  public synchronized void startDocument(int numVectorFields) throws IOException {
+    mainWriter.startDocument(numVectorFields);
+    for (TermVectorsWriter w : writers.values()) {
+      w.startDocument(numVectorFields);
+    }
+  }
+
+  @Override
+  public synchronized void startField(FieldInfo info, int numTerms, boolean positions,
+          boolean offsets) throws IOException {
+    mainWriter.startField(info, numTerms, positions, offsets);
+    for (TermVectorsWriter w : writers.values()) {
+      w.startField(info, numTerms, positions, offsets);
+    }
+  }
+
+  @Override
+  public synchronized void startTerm(BytesRef term, int freq) throws IOException {
+    mainWriter.startTerm(term, freq);
+    for (TermVectorsWriter w : writers.values()) {
+      w.startTerm(term, freq);
+    }
+  }
+
+  @Override
+  public synchronized void addPosition(int position, int startOffset, int endOffset)
+          throws IOException {
+    mainWriter.addPosition(position, startOffset, endOffset);
+    for (TermVectorsWriter w : writers.values()) {
+      w.addPosition(position, startOffset, endOffset);
+    }
+  }
+
+  @Override
+  public synchronized void abort() {
+    mainWriter.abort();
+    for (TermVectorsWriter w : writers.values()) {
+      w.abort();
+    }
+  }
+
+  @Override
+  public synchronized void finish(int numDocs) throws IOException {
+    mainWriter.finish(numDocs);
+    for (TermVectorsWriter w : writers.values()) {
+      w.finish(numDocs);
+    }
+  }
+
+  @Override
+  public Comparator<BytesRef> getComparator() throws IOException {
+    return mainWriter.getComparator();
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java	(working copy)
@@ -0,0 +1,73 @@
+package org.apache.lucene.codecs.tee.impl;
+
+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.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
+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.index.PerDocWriteState;
+import org.apache.lucene.store.Directory;
+
+public class TeePerDocConsumer extends PerDocConsumer {
+  PerDocConsumer mainConsumer;
+  Map<Directory,PerDocConsumer> consumers;
+
+  public TeePerDocConsumer(PerDocConsumer mainConsumer,
+          Map<Directory,PerDocConsumer> consumers) throws IOException {
+    this.mainConsumer = mainConsumer;
+    this.consumers = consumers;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    mainConsumer.close();
+    for (Entry<Directory,PerDocConsumer> e : consumers.entrySet()) {
+      e.getValue().close();
+    }
+  }
+
+  @Override
+  public synchronized DocValuesConsumer addValuesField(Type type, FieldInfo field)
+          throws IOException {
+    return new TeeDocValuesConsumer(mainConsumer, consumers, type, field);
+  }
+
+  @Override
+  public synchronized void abort() {
+    mainConsumer.abort();
+    for (Entry<Directory,PerDocConsumer> e : consumers.entrySet()) {
+      e.getValue().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) {
+    return mainConsumer.canMerge(info);
+  }
+
+  @Override
+  public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException {
+    // TODO Auto-generated method stub
+    return mainConsumer.getDocValuesForMerge(reader, info);
+  }
+
+  @Override
+  public Type getDocValuesType(FieldInfo info) {
+    // TODO Auto-generated method stub
+    return mainConsumer.getDocValuesType(info);
+  } 
+  
+  
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java	(working copy)
@@ -0,0 +1,43 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+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.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+
+public class TeeFieldInfosFormat extends FieldInfosFormat {
+  Codec main;
+  FieldInfosFormat mainFormat;
+  Map<Directory,Codec> outputs;
+  
+  public TeeFieldInfosFormat(Codec main, Map<Directory,Codec> outputs) {
+    this.main = main;
+    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);
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    mainFormat.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java	(working copy)
@@ -0,0 +1,60 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+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.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.store.Directory;
+
+public class TeeDocValuesFormat extends DocValuesFormat {
+  Codec main;
+  DocValuesFormat mainFormat;
+  Map<Directory,Codec> outputs;
+  
+  public TeeDocValuesFormat(Codec main, Map<Directory,Codec> outputs) {
+    this.main = main;
+    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<Directory,PerDocConsumer> consumers = new HashMap<Directory, PerDocConsumer>();
+    for (Entry<Directory,Codec> e : outputs.entrySet()) {
+      PerDocWriteState s = new PerDocWriteState(state.infoStream,
+              e.getKey(), state.segmentName, state.fieldInfos, 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);
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    mainFormat.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java	(working copy)
@@ -0,0 +1,52 @@
+package org.apache.lucene.codecs.tee.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+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 {
+  Codec main;
+  StoredFieldsFormat mainFormat;
+  Map<Directory,Codec> outputs;
+  
+  public TeeStoredFieldsFormat(Codec main, Map<Directory,Codec> outputs) {
+    this.main = main;
+    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, String segment,
+          IOContext context) throws IOException {
+    StoredFieldsWriter mainWriter = mainFormat.fieldsWriter(directory, segment, context);
+    HashMap<Directory,StoredFieldsWriter> writers = new HashMap<Directory,StoredFieldsWriter>();
+    for (Entry<Directory,Codec> e : outputs.entrySet()) {
+      writers.put(e.getKey(), e.getValue().storedFieldsFormat().fieldsWriter(e.getKey(), segment, context));
+    }
+    return new TeeStoredFieldsWriter(mainWriter, writers);
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files) throws IOException {
+    mainFormat.files(info, files);
+  }
+
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java	(working copy)
@@ -0,0 +1,126 @@
+package org.apache.lucene.codecs.tee;
+
+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;
+
+public class TeeDirectory extends Directory {
+  private Set<String> suffixes;
+  private Collection<Directory> dirs;
+  private Directory main;
+  private boolean debug;
+  
+  public TeeDirectory(Directory main, Collection<Directory> dirs, Set<String> suffixes, boolean debug) {
+    this.main = main;
+    this.lockFactory = main.getLockFactory();
+    this.dirs = dirs;
+    if (suffixes == null || suffixes.isEmpty()) {
+      this.suffixes = Collections.emptySet();
+    } else {
+      this.suffixes = suffixes;
+    }
+    this.debug = debug;
+  }
+  
+  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<Directory,IndexOutput> outs = new HashMap<Directory,IndexOutput>();
+      outs.put(main, mainOutput);
+      for (Directory d : dirs) {
+        outs.put(d, d.createOutput(name, context));
+      }
+      return new TeeIndexOutput(null, outs, null);
+    } else {
+      return mainOutput;
+    }
+  }
+
+  @Override
+  public synchronized void sync(Collection<String> names) throws IOException {
+    main.sync(names);
+    if (suffixes.isEmpty()) {
+      for (Directory d : dirs) {
+        d.sync(names);
+      }
+    } else {
+      Set<String> syncNames = new HashSet<String>();
+      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/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeCodec.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeCodec.java	(revision 0)
+++ lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeCodec.java	(working copy)
@@ -0,0 +1,95 @@
+package org.apache.lucene.codecs.tee;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+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.SegmentInfosFormat;
+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;
+
+public class TeeCodec extends Codec {
+  public Map<Directory,Codec> outputs;
+  public Codec mainCodec;
+  
+  private final PostingsFormat postingsFormat;
+  private final DocValuesFormat docValuesFormat;
+  private final FieldInfosFormat fieldInfosFormat;
+  private final LiveDocsFormat liveDocsFormat;
+  private final NormsFormat normsFormat;
+  private final SegmentInfosFormat segmentInfosFormat;
+  private final StoredFieldsFormat storedFieldsFormat;
+  private final TermVectorsFormat termVectorsFormat;
+  
+  public TeeCodec(Codec mainCodec, Map<Directory,Codec> outputs) {
+    super(mainCodec.getName());
+    this.mainCodec = mainCodec;
+    this.outputs = outputs;
+    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);
+    segmentInfosFormat = 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 SegmentInfosFormat segmentInfosFormat() {
+    return segmentInfosFormat;
+  }
+
+  @Override
+  public NormsFormat normsFormat() {
+    return normsFormat;
+  }
+
+  @Override
+  public LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+}

Property changes on: lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeCodec.java
___________________________________________________________________
Added: svn:eol-style
## -0,0 +1 ##
+native
Index: lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java
===================================================================
--- lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java	(revision 1244135)
+++ lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java	(working copy)
@@ -79,15 +79,20 @@
     }
   }
   
+  // nocommit
   @Override
-  protected boolean canMerge(FieldInfo info) {
+  public boolean canMerge(FieldInfo info) {
     return info.normsPresent();
   }
 
+  // nocommit
   @Override
-  protected Type getDocValuesType(FieldInfo info) {
+  public Type getDocValuesType(FieldInfo info) {
     return info.getNormType();
   }
+  
+  //nocommit
+  // how is this actually working when it does not override getValuesForMerge?!
 
   @Override
   public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
