You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2010/11/30 12:22:46 UTC

svn commit: r1040463 [2/8] - in /lucene/dev/trunk: lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ lucene/contrib/db/bdb-je/src/java/org/apache/lucene/store/je/ lucene/contrib/db/bdb/src/java/org/apache/lucene/store/db/ luce...

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/DateTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/DateTools.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/DateTools.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/DateTools.java Tue Nov 30 11:22:39 2010
@@ -36,10 +36,6 @@ import org.apache.lucene.util.NumericUti
  * save dates with a finer resolution than you really need, as then
  * RangeQuery and PrefixQuery will require more memory and become slower.
  * 
- * <P>Compared to {@link DateField} the strings generated by the methods
- * in this class take slightly more space, unless your selected resolution
- * is set to <code>Resolution.DAY</code> or lower.
- *
  * <P>
  * Another approach is {@link NumericUtils}, which provides
  * a sortable binary representation (prefix encoded) of numeric values, which

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/Field.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/Field.java Tue Nov 30 11:22:39 2010
@@ -17,13 +17,13 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.index.IndexWriter;   // for javadoc
-import org.apache.lucene.util.StringHelper;
-
 import java.io.Reader;
 import java.io.Serializable;
 
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.util.StringHelper;
+
 /**
   A field is a section of a Document.  Each field has two parts, a name and a
   value.  Values may be free text, provided as a String or as a Reader, or they
@@ -521,24 +521,6 @@ public final class Field extends Abstrac
    * 
    * @param name The name of the field
    * @param value The binary value
-   * @param store Must be Store.YES
-   * @throws IllegalArgumentException if store is <code>Store.NO</code> 
-   * @deprecated Use {@link #Field(String, byte[]) instead}
-   */
-  @Deprecated
-  public Field(String name, byte[] value, Store store) {
-    this(name, value, 0, value.length);
-
-    if (store == Store.NO) {
-      throw new IllegalArgumentException("binary values can't be unstored");
-    }
-  }
-
-  /**
-   * Create a stored field with binary value. Optionally the value may be compressed.
-   * 
-   * @param name The name of the field
-   * @param value The binary value
    */
   public Field(String name, byte[] value) {
     this(name, value, 0, value.length);
@@ -551,26 +533,6 @@ public final class Field extends Abstrac
    * @param value The binary value
    * @param offset Starting offset in value where this Field's bytes are
    * @param length Number of bytes to use for this Field, starting at offset
-   * @param store How <code>value</code> should be stored (compressed or not)
-   * @throws IllegalArgumentException if store is <code>Store.NO</code> 
-   * @deprecated Use {@link #Field(String, byte[], int, int) instead}
-   */
-  @Deprecated
-  public Field(String name, byte[] value, int offset, int length, Store store) {
-    this(name, value, offset, length);
-
-    if (store == Store.NO) {
-      throw new IllegalArgumentException("binary values can't be unstored");
-    }
-  }
-
-  /**
-   * Create a stored field with binary value. Optionally the value may be compressed.
-   * 
-   * @param name The name of the field
-   * @param value The binary value
-   * @param offset Starting offset in value where this Field's bytes are
-   * @param length Number of bytes to use for this Field, starting at offset
    */
   public Field(String name, byte[] value, int offset, int length) {
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java Tue Nov 30 11:22:39 2010
@@ -23,6 +23,7 @@ import org.apache.lucene.store.BufferedI
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.Lock;
 
+import java.util.Collection;
 import java.util.HashMap;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -227,6 +228,10 @@ public class CompoundFileReader extends 
         throw new UnsupportedOperationException();
     }
 
+    @Override
+    public void sync(Collection<String> names) throws IOException {
+    }
+
     /** Not implemented
      * @throws UnsupportedOperationException */
     @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Tue Nov 30 11:22:39 2010
@@ -180,69 +180,12 @@ import java.util.Date;
  * keeps track of the last non commit checkpoint.
  */
 public class IndexWriter implements Closeable {
-
-  /**
-   * Default value for the write lock timeout (1,000).
-   * @see #setDefaultWriteLockTimeout
-   * @deprecated use {@link IndexWriterConfig#WRITE_LOCK_TIMEOUT} instead
-   */
-  @Deprecated
-  public static long WRITE_LOCK_TIMEOUT = IndexWriterConfig.WRITE_LOCK_TIMEOUT;
-
-  private long writeLockTimeout;
-
   /**
    * Name of the write lock in the index.
    */
   public static final String WRITE_LOCK_NAME = "write.lock";
 
   /**
-   * Value to denote a flush trigger is disabled
-   * @deprecated use {@link IndexWriterConfig#DISABLE_AUTO_FLUSH} instead
-   */
-  @Deprecated
-  public final static int DISABLE_AUTO_FLUSH = IndexWriterConfig.DISABLE_AUTO_FLUSH;
-
-  /**
-   * Disabled by default (because IndexWriter flushes by RAM usage
-   * by default). Change using {@link #setMaxBufferedDocs(int)}.
-   * @deprecated use {@link IndexWriterConfig#DEFAULT_MAX_BUFFERED_DOCS} instead.
-   */
-  @Deprecated
-  public final static int DEFAULT_MAX_BUFFERED_DOCS = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
-
-  /**
-   * Default value is 16 MB (which means flush when buffered
-   * docs consume 16 MB RAM).  Change using {@link #setRAMBufferSizeMB}.
-   * @deprecated use {@link IndexWriterConfig#DEFAULT_RAM_BUFFER_SIZE_MB} instead.
-   */
-  @Deprecated
-  public final static double DEFAULT_RAM_BUFFER_SIZE_MB = IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB;
-
-  /**
-   * Disabled by default (because IndexWriter flushes by RAM usage
-   * by default). Change using {@link #setMaxBufferedDeleteTerms(int)}.
-   * @deprecated use {@link IndexWriterConfig#DEFAULT_MAX_BUFFERED_DELETE_TERMS} instead
-   */
-  @Deprecated
-  public final static int DEFAULT_MAX_BUFFERED_DELETE_TERMS = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS;
-
-  /**
-   * Default value is 10,000. Change using {@link #setMaxFieldLength(int)}.
-   * 
-   * @deprecated see {@link IndexWriterConfig}
-   */
-  @Deprecated
-  public final static int DEFAULT_MAX_FIELD_LENGTH = 10000;
-
-  /**
-   * Default value is 128. Change using {@link #setTermIndexInterval(int)}.
-   * @deprecated use {@link IndexWriterConfig#DEFAULT_TERM_INDEX_INTERVAL} instead.
-   */
-  @Deprecated
-  public final static int DEFAULT_TERM_INDEX_INTERVAL = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
-
-  /**
    * Absolute hard maximum length for a term, in bytes once
    * encoded as UTF8.  If a term arrives from the analyzer
    * longer than this length, it is skipped and a message is
@@ -268,9 +211,6 @@ public class IndexWriter implements Clos
   private final Directory directory;  // where this index resides
   private final Analyzer analyzer;    // how to analyze text
 
-  // TODO 4.0: this should be made final once the setter is out
-  private /*final*/Similarity similarity = Similarity.getDefault(); // how to normalize
-
   private volatile long changeCount; // increments every time a change is completed
   private long lastCommitChangeCount; // last changeCount that was committed
 
@@ -290,8 +230,7 @@ public class IndexWriter implements Clos
 
   private Lock writeLock;
 
-  // TODO 4.0: this should be made final once the setter is out
-  private /*final*/int termIndexInterval;
+  private final int termIndexInterval;
 
   private boolean closed;
   private boolean closing;
@@ -301,8 +240,7 @@ public class IndexWriter implements Clos
   private HashSet<SegmentInfo> mergingSegments = new HashSet<SegmentInfo>();
 
   private MergePolicy mergePolicy;
-  // TODO 4.0: this should be made final once the setter is removed
-  private /*final*/MergeScheduler mergeScheduler;
+  private final MergeScheduler mergeScheduler;
   private LinkedList<MergePolicy.OneMerge> pendingMerges = new LinkedList<MergePolicy.OneMerge>();
   private Set<MergePolicy.OneMerge> runningMerges = new HashSet<MergePolicy.OneMerge>();
   private List<MergePolicy.OneMerge> mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
@@ -730,276 +668,6 @@ public class IndexWriter implements Clos
       throw new IllegalArgumentException("this method can only be called when the merge policy is the default LogMergePolicy");
   }
 
-  /** <p>Get the current setting of whether newly flushed
-   *  segments will use the compound file format.  Note that
-   *  this just returns the value previously set with
-   *  setUseCompoundFile(boolean), or the default value
-   *  (true).  You cannot use this to query the status of
-   *  previously flushed segments.</p>
-   *
-   *  <p>Note that this method is a convenience method: it
-   *  just calls mergePolicy.getUseCompoundFile as long as
-   *  mergePolicy is an instance of {@link LogMergePolicy}.
-   *  Otherwise an IllegalArgumentException is thrown.</p>
-   *
-   *  @see #setUseCompoundFile(boolean)
-   *  @deprecated use {@link LogMergePolicy#getUseCompoundDocStore()} and
-   *  {@link LogMergePolicy#getUseCompoundFile()} directly.
-   */
-  @Deprecated
-  public boolean getUseCompoundFile() {
-    return getLogMergePolicy().getUseCompoundFile();
-  }
-
-  /**
-   * <p>
-   * Setting to turn on usage of a compound file. When on, multiple files for
-   * each segment are merged into a single file when a new segment is flushed.
-   * </p>
-   * 
-   * <p>
-   * Note that this method is a convenience method: it just calls
-   * mergePolicy.setUseCompoundFile as long as mergePolicy is an instance of
-   * {@link LogMergePolicy}. Otherwise an IllegalArgumentException is thrown.
-   * </p>
-   * 
-   * @deprecated use {@link LogMergePolicy#setUseCompoundDocStore(boolean)} and
-   *             {@link LogMergePolicy#setUseCompoundFile(boolean)} directly.
-   *             Note that this method set the given value on both, therefore
-   *             you should consider doing the same.
-   */
-  @Deprecated
-  public void setUseCompoundFile(boolean value) {
-    getLogMergePolicy().setUseCompoundFile(value);
-    getLogMergePolicy().setUseCompoundDocStore(value);
-  }
-
-  /** Expert: Set the Similarity implementation used by this IndexWriter.
-   *
-   * @see Similarity#setDefault(Similarity)
-   * @deprecated use {@link IndexWriterConfig#setSimilarity(Similarity)} instead
-   */
-  @Deprecated
-  public void setSimilarity(Similarity similarity) {
-    ensureOpen();
-    this.similarity = similarity;
-    docWriter.setSimilarity(similarity);
-    // Required so config.getSimilarity returns the right value. But this will
-    // go away together with the method in 4.0.
-    config.setSimilarity(similarity);
-  }
-
-  /** Expert: Return the Similarity implementation used by this IndexWriter.
-   *
-   * <p>This defaults to the current value of {@link Similarity#getDefault()}.
-   * @deprecated use {@link IndexWriterConfig#getSimilarity()} instead
-   */
-  @Deprecated
-  public Similarity getSimilarity() {
-    ensureOpen();
-    return similarity;
-  }
-
-  /** Expert: Set the interval between indexed terms.  Large values cause less
-   * memory to be used by IndexReader, but slow random-access to terms.  Small
-   * values cause more memory to be used by an IndexReader, and speed
-   * random-access to terms.
-   *
-   * This parameter determines the amount of computation required per query
-   * term, regardless of the number of documents that contain that term.  In
-   * particular, it is the maximum number of other terms that must be
-   * scanned before a term is located and its frequency and position information
-   * may be processed.  In a large index with user-entered query terms, query
-   * processing time is likely to be dominated not by term lookup but rather
-   * by the processing of frequency and positional data.  In a small index
-   * or when many uncommon query terms are generated (e.g., by wildcard
-   * queries) term lookup may become a dominant cost.
-   *
-   * In particular, <code>numUniqueTerms/interval</code> terms are read into
-   * memory by an IndexReader, and, on average, <code>interval/2</code> terms
-   * must be scanned for each random term access.
-   *
-   * @see #DEFAULT_TERM_INDEX_INTERVAL
-   * @deprecated use {@link IndexWriterConfig#setTermIndexInterval(int)}
-   */
-  @Deprecated
-  public void setTermIndexInterval(int interval) {
-    ensureOpen();
-    this.termIndexInterval = interval;
-    // Required so config.getTermIndexInterval returns the right value. But this
-    // will go away together with the method in 4.0.
-    config.setTermIndexInterval(interval);
-  }
-
-  /** Expert: Return the interval between indexed terms.
-   *
-   * @see #setTermIndexInterval(int)
-   * @deprecated use {@link IndexWriterConfig#getTermIndexInterval()}
-   */
-  @Deprecated
-  public int getTermIndexInterval() {
-    // We pass false because this method is called by SegmentMerger while we are in the process of closing
-    ensureOpen(false);
-    return termIndexInterval;
-  }
-
-  /**
-   * Constructs an IndexWriter for the index in <code>d</code>.
-   * Text will be analyzed with <code>a</code>.  If <code>create</code>
-   * is true, then a new, empty index will be created in
-   * <code>d</code>, replacing the index already there, if any.
-   *
-   * @param d the index directory
-   * @param a the analyzer to use
-   * @param create <code>true</code> to create the index or overwrite
-   *  the existing one; <code>false</code> to append to the existing
-   *  index
-   * @param mfl Maximum field length in number of terms/tokens: LIMITED, UNLIMITED, or user-specified
-   *   via the MaxFieldLength constructor.
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws LockObtainFailedException if another writer
-   *  has this index open (<code>write.lock</code> could not
-   *  be obtained)
-   * @throws IOException if the directory cannot be read/written to, or
-   *  if it does not exist and <code>create</code> is
-   *  <code>false</code> or if there is any other low-level
-   *  IO error
-   *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
-   */
-  @Deprecated
-  public IndexWriter(Directory d, Analyzer a, boolean create, MaxFieldLength mfl)
-       throws CorruptIndexException, LockObtainFailedException, IOException {
-    this(d, new IndexWriterConfig(Version.LUCENE_31, a).setOpenMode(
-        create ? OpenMode.CREATE : OpenMode.APPEND).setMaxFieldLength(
-        mfl.getLimit()));
-  }
-
-  /**
-   * Constructs an IndexWriter for the index in
-   * <code>d</code>, first creating it if it does not
-   * already exist.  Text will be analyzed with
-   * <code>a</code>.
-   *
-   * @param d the index directory
-   * @param a the analyzer to use
-   * @param mfl Maximum field length in number of terms/tokens: LIMITED, UNLIMITED, or user-specified
-   *   via the MaxFieldLength constructor.
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws LockObtainFailedException if another writer
-   *  has this index open (<code>write.lock</code> could not
-   *  be obtained)
-   * @throws IOException if the directory cannot be
-   *  read/written to or if there is any other low-level
-   *  IO error
-   *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
-   */
-  @Deprecated
-  public IndexWriter(Directory d, Analyzer a, MaxFieldLength mfl)
-    throws CorruptIndexException, LockObtainFailedException, IOException {
-    this(d, new IndexWriterConfig(Version.LUCENE_31, a)
-        .setMaxFieldLength(mfl.getLimit()));
-  }
-
-  /**
-   * Expert: constructs an IndexWriter with a custom {@link
-   * IndexDeletionPolicy}, for the index in <code>d</code>,
-   * first creating it if it does not already exist.  Text
-   * will be analyzed with <code>a</code>.
-   *
-   * @param d the index directory
-   * @param a the analyzer to use
-   * @param deletionPolicy see <a href="#deletionPolicy">above</a>
-   * @param mfl whether or not to limit field lengths
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws LockObtainFailedException if another writer
-   *  has this index open (<code>write.lock</code> could not
-   *  be obtained)
-   * @throws IOException if the directory cannot be
-   *  read/written to or if there is any other low-level
-   *  IO error
-   *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
-   */
-  @Deprecated
-  public IndexWriter(Directory d, Analyzer a, IndexDeletionPolicy deletionPolicy, MaxFieldLength mfl)
-    throws CorruptIndexException, LockObtainFailedException, IOException {
-    this(d, new IndexWriterConfig(Version.LUCENE_31, a).setMaxFieldLength(
-        mfl.getLimit()).setIndexDeletionPolicy(deletionPolicy));
-  }
-
-  /**
-   * Expert: constructs an IndexWriter with a custom {@link
-   * IndexDeletionPolicy}, for the index in <code>d</code>.
-   * Text will be analyzed with <code>a</code>.  If
-   * <code>create</code> is true, then a new, empty index
-   * will be created in <code>d</code>, replacing the index
-   * already there, if any.
-   *
-   * @param d the index directory
-   * @param a the analyzer to use
-   * @param create <code>true</code> to create the index or overwrite
-   *  the existing one; <code>false</code> to append to the existing
-   *  index
-   * @param deletionPolicy see <a href="#deletionPolicy">above</a>
-   * @param mfl {@link org.apache.lucene.index.IndexWriter.MaxFieldLength}, whether or not to limit field lengths.  Value is in number of terms/tokens
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws LockObtainFailedException if another writer
-   *  has this index open (<code>write.lock</code> could not
-   *  be obtained)
-   * @throws IOException if the directory cannot be read/written to, or
-   *  if it does not exist and <code>create</code> is
-   *  <code>false</code> or if there is any other low-level
-   *  IO error
-   *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
-   */
-  @Deprecated
-  public IndexWriter(Directory d, Analyzer a, boolean create, IndexDeletionPolicy deletionPolicy, MaxFieldLength mfl)
-       throws CorruptIndexException, LockObtainFailedException, IOException {
-    this(d, new IndexWriterConfig(Version.LUCENE_31, a).setOpenMode(
-        create ? OpenMode.CREATE : OpenMode.APPEND).setMaxFieldLength(
-        mfl.getLimit()).setIndexDeletionPolicy(deletionPolicy));
-  }
-  
-  /**
-   * Expert: constructs an IndexWriter on specific commit
-   * point, with a custom {@link IndexDeletionPolicy}, for
-   * the index in <code>d</code>.  Text will be analyzed
-   * with <code>a</code>.
-   *
-   * <p> This is only meaningful if you've used a {@link
-   * IndexDeletionPolicy} in that past that keeps more than
-   * just the last commit.
-   * 
-   * <p>This operation is similar to {@link #rollback()},
-   * except that method can only rollback what's been done
-   * with the current instance of IndexWriter since its last
-   * commit, whereas this method can rollback to an
-   * arbitrary commit point from the past, assuming the
-   * {@link IndexDeletionPolicy} has preserved past
-   * commits.
-   *
-   * @param d the index directory
-   * @param a the analyzer to use
-   * @param deletionPolicy see <a href="#deletionPolicy">above</a>
-   * @param mfl whether or not to limit field lengths, value is in number of terms/tokens.  See {@link org.apache.lucene.index.IndexWriter.MaxFieldLength}.
-   * @param commit which commit to open
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws LockObtainFailedException if another writer
-   *  has this index open (<code>write.lock</code> could not
-   *  be obtained)
-   * @throws IOException if the directory cannot be read/written to, or
-   *  if it does not exist and <code>create</code> is
-   *  <code>false</code> or if there is any other low-level
-   *  IO error
-   *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
-   */
-  @Deprecated
-  public IndexWriter(Directory d, Analyzer a, IndexDeletionPolicy deletionPolicy, MaxFieldLength mfl, IndexCommit commit)
-       throws CorruptIndexException, LockObtainFailedException, IOException {
-    this(d, new IndexWriterConfig(Version.LUCENE_31, a)
-        .setOpenMode(OpenMode.APPEND).setMaxFieldLength(mfl.getLimit())
-        .setIndexDeletionPolicy(deletionPolicy).setIndexCommit(commit));
-  }
-  
   CodecProvider codecs;
 
   /**
@@ -1038,8 +706,6 @@ public class IndexWriter implements Clos
     setMessageID(defaultInfoStream);
     maxFieldLength = conf.getMaxFieldLength();
     termIndexInterval = conf.getTermIndexInterval();
-    writeLockTimeout = conf.getWriteLockTimeout();
-    similarity = conf.getSimilarity();
     mergePolicy = conf.getMergePolicy();
     mergePolicy.setIndexWriter(this);
     mergeScheduler = conf.getMergeScheduler();
@@ -1061,7 +727,7 @@ public class IndexWriter implements Clos
 
     writeLock = directory.makeLock(WRITE_LOCK_NAME);
 
-    if (!writeLock.obtain(writeLockTimeout)) // obtain write lock
+    if (!writeLock.obtain(conf.getWriteLockTimeout())) // obtain write lock
       throw new LockObtainFailedException("Index locked for write: " + writeLock);
 
     boolean success = false;
@@ -1177,202 +843,12 @@ public class IndexWriter implements Clos
   }
   
   /**
-   * Expert: set the merge policy used by this writer.
-   * 
-   * @deprecated use {@link IndexWriterConfig#setMergePolicy(MergePolicy)} instead.
-   */
-  @Deprecated
-  public void setMergePolicy(MergePolicy mp) {
-    ensureOpen();
-    if (mp == null)
-      throw new NullPointerException("MergePolicy must be non-null");
-
-    if (mergePolicy != mp)
-      mergePolicy.close();
-    mergePolicy = mp;
-    mergePolicy.setIndexWriter(this);
-    pushMaxBufferedDocs();
-    if (infoStream != null)
-      message("setMergePolicy " + mp);
-    // Required so config.getMergePolicy returns the right value. But this will
-    // go away together with the method in 4.0.
-    config.setMergePolicy(mp);
-  }
-
-  /**
-   * Expert: returns the current MergePolicy in use by this writer.
-   * @see #setMergePolicy
-   * 
-   * @deprecated use {@link IndexWriterConfig#getMergePolicy()} instead
-   */
-  @Deprecated
-  public MergePolicy getMergePolicy() {
-    ensureOpen();
-    return mergePolicy;
-  }
-
-  /**
-   * Expert: set the merge scheduler used by this writer.
-   * @deprecated use {@link IndexWriterConfig#setMergeScheduler(MergeScheduler)} instead
-   */
-  @Deprecated
-  synchronized public void setMergeScheduler(MergeScheduler mergeScheduler) throws CorruptIndexException, IOException {
-    ensureOpen();
-    if (mergeScheduler == null)
-      throw new NullPointerException("MergeScheduler must be non-null");
-
-    if (this.mergeScheduler != mergeScheduler) {
-      finishMerges(true);
-      this.mergeScheduler.close();
-    }
-    this.mergeScheduler = mergeScheduler;
-    if (infoStream != null)
-      message("setMergeScheduler " + mergeScheduler);
-    // Required so config.getMergeScheduler returns the right value. But this will
-    // go away together with the method in 4.0.
-    config.setMergeScheduler(mergeScheduler);
-  }
-
-  /**
-   * Expert: returns the current MergeScheduler in use by this
-   * writer.
-   * @see #setMergeScheduler(MergeScheduler)
-   * @deprecated use {@link IndexWriterConfig#getMergeScheduler()} instead
-   */
-  @Deprecated
-  public MergeScheduler getMergeScheduler() {
-    ensureOpen();
-    return mergeScheduler;
-  }
-
-  /** <p>Determines the largest segment (measured by
-   * document count) that may be merged with other segments.
-   * Small values (e.g., less than 10,000) are best for
-   * interactive indexing, as this limits the length of
-   * pauses while indexing to a few seconds.  Larger values
-   * are best for batched indexing and speedier
-   * searches.</p>
-   *
-   * <p>The default value is {@link Integer#MAX_VALUE}.</p>
-   *
-   * <p>Note that this method is a convenience method: it
-   * just calls mergePolicy.setMaxMergeDocs as long as
-   * mergePolicy is an instance of {@link LogMergePolicy}.
-   * Otherwise an IllegalArgumentException is thrown.</p>
-   *
-   * <p>The default merge policy ({@link
-   * LogByteSizeMergePolicy}) also allows you to set this
-   * limit by net size (in MB) of the segment, using {@link
-   * LogByteSizeMergePolicy#setMaxMergeMB}.</p>
-   * @deprecated use {@link LogMergePolicy#setMaxMergeDocs(int)} directly.
-   */
-  @Deprecated
-  public void setMaxMergeDocs(int maxMergeDocs) {
-    getLogMergePolicy().setMaxMergeDocs(maxMergeDocs);
-  }
-
-  /**
-   * <p>Returns the largest segment (measured by document
-   * count) that may be merged with other segments.</p>
-   *
-   * <p>Note that this method is a convenience method: it
-   * just calls mergePolicy.getMaxMergeDocs as long as
-   * mergePolicy is an instance of {@link LogMergePolicy}.
-   * Otherwise an IllegalArgumentException is thrown.</p>
-   *
-   * @see #setMaxMergeDocs
-   * @deprecated use {@link LogMergePolicy#getMaxMergeDocs()} directly.
-   */
-  @Deprecated
-  public int getMaxMergeDocs() {
-    return getLogMergePolicy().getMaxMergeDocs();
-  }
-
-  /**
-   * The maximum number of terms that will be indexed for a single field in a
-   * document.  This limits the amount of memory required for indexing, so that
-   * collections with very large files will not crash the indexing process by
-   * running out of memory.  This setting refers to the number of running terms,
-   * not to the number of different terms.<p/>
-   * <strong>Note:</strong> this silently truncates large documents, excluding from the
-   * index all terms that occur further in the document.  If you know your source
-   * documents are large, be sure to set this value high enough to accomodate
-   * the expected size.  If you set it to Integer.MAX_VALUE, then the only limit
-   * is your memory, but you should anticipate an OutOfMemoryError.<p/>
-   * By default, no more than {@link #DEFAULT_MAX_FIELD_LENGTH} terms
-   * will be indexed for a field.
-   * @deprecated use {@link IndexWriterConfig#setMaxFieldLength(int)} instead
-   */
-  @Deprecated
-  public void setMaxFieldLength(int maxFieldLength) {
-    ensureOpen();
-    this.maxFieldLength = maxFieldLength;
-    docWriter.setMaxFieldLength(maxFieldLength);
-    if (infoStream != null)
-      message("setMaxFieldLength " + maxFieldLength);
-    // Required so config.getMaxFieldLength returns the right value. But this
-    // will go away together with the method in 4.0.
-    config.setMaxFieldLength(maxFieldLength);
-  }
-
-  /**
-   * Returns the maximum number of terms that will be
-   * indexed for a single field in a document.
-   * @see #setMaxFieldLength
-   * @deprecated use {@link IndexWriterConfig#getMaxFieldLength()} instead
-   */
-  @Deprecated
-  public int getMaxFieldLength() {
-    ensureOpen();
-    return maxFieldLength;
-  }
-
-  /** Determines the minimal number of documents required
-   * before the buffered in-memory documents are flushed as
-   * a new Segment.  Large values generally gives faster
-   * indexing.
-   *
-   * <p>When this is set, the writer will flush every
-   * maxBufferedDocs added documents.  Pass in {@link
-   * #DISABLE_AUTO_FLUSH} to prevent triggering a flush due
-   * to number of buffered documents.  Note that if flushing
-   * by RAM usage is also enabled, then the flush will be
-   * triggered by whichever comes first.</p>
-   *
-   * <p>Disabled by default (writer flushes by RAM usage).</p>
-   *
-   * @throws IllegalArgumentException if maxBufferedDocs is
-   * enabled but smaller than 2, or it disables maxBufferedDocs
-   * when ramBufferSize is already disabled
-   * @see #setRAMBufferSizeMB
-   * @deprecated use {@link IndexWriterConfig#setMaxBufferedDocs(int)} instead.
-   */
-  @Deprecated
-  public void setMaxBufferedDocs(int maxBufferedDocs) {
-    ensureOpen();
-    if (maxBufferedDocs != DISABLE_AUTO_FLUSH && maxBufferedDocs < 2)
-      throw new IllegalArgumentException(
-          "maxBufferedDocs must at least be 2 when enabled");
-    if (maxBufferedDocs == DISABLE_AUTO_FLUSH
-        && getRAMBufferSizeMB() == DISABLE_AUTO_FLUSH)
-      throw new IllegalArgumentException(
-          "at least one of ramBufferSize and maxBufferedDocs must be enabled");
-    docWriter.setMaxBufferedDocs(maxBufferedDocs);
-    pushMaxBufferedDocs();
-    if (infoStream != null)
-      message("setMaxBufferedDocs " + maxBufferedDocs);
-    // Required so config.getMaxBufferedDocs returns the right value. But this
-    // will go away together with the method in 4.0.
-    config.setMaxBufferedDocs(maxBufferedDocs);
-  }
-
-  /**
    * If we are flushing by doc count (not by RAM usage), and
    * using LogDocMergePolicy then push maxBufferedDocs down
    * as its minMergeDocs, to keep backwards compatibility.
    */
   private void pushMaxBufferedDocs() {
-    if (docWriter.getMaxBufferedDocs() != DISABLE_AUTO_FLUSH) {
+    if (docWriter.getMaxBufferedDocs() != IndexWriterConfig.DISABLE_AUTO_FLUSH) {
       final MergePolicy mp = mergePolicy;
       if (mp instanceof LogDocMergePolicy) {
         LogDocMergePolicy lmp = (LogDocMergePolicy) mp;
@@ -1386,164 +862,6 @@ public class IndexWriter implements Clos
     }
   }
 
-  /**
-   * Returns the number of buffered added documents that will
-   * trigger a flush if enabled.
-   * @see #setMaxBufferedDocs
-   * @deprecated use {@link IndexWriterConfig#getMaxBufferedDocs()} instead.
-   */
-  @Deprecated
-  public int getMaxBufferedDocs() {
-    ensureOpen();
-    return docWriter.getMaxBufferedDocs();
-  }
-
-  /** Determines the amount of RAM that may be used for
-   * buffering added documents and deletions before they are
-   * flushed to the Directory.  Generally for faster
-   * indexing performance it's best to flush by RAM usage
-   * instead of document count and use as large a RAM buffer
-   * as you can.
-   *
-   * <p>When this is set, the writer will flush whenever
-   * buffered documents and deletions use this much RAM.
-   * Pass in {@link #DISABLE_AUTO_FLUSH} to prevent
-   * triggering a flush due to RAM usage.  Note that if
-   * flushing by document count is also enabled, then the
-   * flush will be triggered by whichever comes first.</p>
-   *
-   * <p> <b>NOTE</b>: the account of RAM usage for pending
-   * deletions is only approximate.  Specifically, if you
-   * delete by Query, Lucene currently has no way to measure
-   * the RAM usage if individual Queries so the accounting
-   * will under-estimate and you should compensate by either
-   * calling commit() periodically yourself, or by using
-   * {@link #setMaxBufferedDeleteTerms} to flush by count
-   * instead of RAM usage (each buffered delete Query counts
-   * as one).
-   *
-   * <p> <b>NOTE</b>: because IndexWriter uses
-   * <code>int</code>s when managing its internal storage,
-   * the absolute maximum value for this setting is somewhat
-   * less than 2048 MB.  The precise limit depends on
-   * various factors, such as how large your documents are,
-   * how many fields have norms, etc., so it's best to set
-   * this value comfortably under 2048.</p>
-   *
-   * <p> The default value is {@link #DEFAULT_RAM_BUFFER_SIZE_MB}.</p>
-   * 
-   * @throws IllegalArgumentException if ramBufferSize is
-   * enabled but non-positive, or it disables ramBufferSize
-   * when maxBufferedDocs is already disabled
-   * @deprecated use {@link IndexWriterConfig#setRAMBufferSizeMB(double)} instead.
-   */
-  @Deprecated
-  public void setRAMBufferSizeMB(double mb) {
-    if (mb > 2048.0) {
-      throw new IllegalArgumentException("ramBufferSize " + mb + " is too large; should be comfortably less than 2048");
-    }
-    if (mb != DISABLE_AUTO_FLUSH && mb <= 0.0)
-      throw new IllegalArgumentException(
-          "ramBufferSize should be > 0.0 MB when enabled");
-    if (mb == DISABLE_AUTO_FLUSH && getMaxBufferedDocs() == DISABLE_AUTO_FLUSH)
-      throw new IllegalArgumentException(
-          "at least one of ramBufferSize and maxBufferedDocs must be enabled");
-    docWriter.setRAMBufferSizeMB(mb);
-    if (infoStream != null)
-      message("setRAMBufferSizeMB " + mb);
-    // Required so config.getRAMBufferSizeMB returns the right value. But this
-    // will go away together with the method in 4.0.
-    config.setRAMBufferSizeMB(mb);
-  }
-
-  /**
-   * Returns the value set by {@link #setRAMBufferSizeMB} if enabled.
-   * @deprecated use {@link IndexWriterConfig#getRAMBufferSizeMB()} instead.
-   */
-  @Deprecated
-  public double getRAMBufferSizeMB() {
-    return docWriter.getRAMBufferSizeMB();
-  }
-
-  /**
-   * <p>Determines the minimal number of delete terms required before the buffered
-   * in-memory delete terms are applied and flushed. If there are documents
-   * buffered in memory at the time, they are merged and a new segment is
-   * created.</p>
-
-   * <p>Disabled by default (writer flushes by RAM usage).</p>
-   * 
-   * @throws IllegalArgumentException if maxBufferedDeleteTerms
-   * is enabled but smaller than 1
-   * @see #setRAMBufferSizeMB
-   * @deprecated use {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)} instead.
-   */
-  @Deprecated
-  public void setMaxBufferedDeleteTerms(int maxBufferedDeleteTerms) {
-    ensureOpen();
-    if (maxBufferedDeleteTerms != DISABLE_AUTO_FLUSH
-        && maxBufferedDeleteTerms < 1)
-      throw new IllegalArgumentException(
-          "maxBufferedDeleteTerms must at least be 1 when enabled");
-    docWriter.setMaxBufferedDeleteTerms(maxBufferedDeleteTerms);
-    if (infoStream != null)
-      message("setMaxBufferedDeleteTerms " + maxBufferedDeleteTerms);
-    // Required so config.getMaxBufferedDeleteTerms returns the right value. But
-    // this will go away together with the method in 4.0.
-    config.setMaxBufferedDeleteTerms(maxBufferedDeleteTerms);
-  }
-
-  /**
-   * Returns the number of buffered deleted terms that will
-   * trigger a flush if enabled.
-   * @see #setMaxBufferedDeleteTerms
-   * @deprecated use {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} instead
-   */
-  @Deprecated
-  public int getMaxBufferedDeleteTerms() {
-    ensureOpen();
-    return docWriter.getMaxBufferedDeleteTerms();
-  }
-
-  /** Determines how often segment indices are merged by addDocument().  With
-   * smaller values, less RAM is used while indexing, and searches on
-   * unoptimized indices are faster, but indexing speed is slower.  With larger
-   * values, more RAM is used during indexing, and while searches on unoptimized
-   * indices are slower, indexing is faster.  Thus larger values (> 10) are best
-   * for batch index creation, and smaller values (< 10) for indices that are
-   * interactively maintained.
-   *
-   * <p>Note that this method is a convenience method: it
-   * just calls mergePolicy.setMergeFactor as long as
-   * mergePolicy is an instance of {@link LogMergePolicy}.
-   * Otherwise an IllegalArgumentException is thrown.</p>
-   *
-   * <p>This must never be less than 2.  The default value is 10.
-   * @deprecated use {@link LogMergePolicy#setMergeFactor(int)} directly.
-   */
-  @Deprecated
-  public void setMergeFactor(int mergeFactor) {
-    getLogMergePolicy().setMergeFactor(mergeFactor);
-  }
-
-  /**
-   * <p>Returns the number of segments that are merged at
-   * once and also controls the total number of segments
-   * allowed to accumulate in the index.</p>
-   *
-   * <p>Note that this method is a convenience method: it
-   * just calls mergePolicy.getMergeFactor as long as
-   * mergePolicy is an instance of {@link LogMergePolicy}.
-   * Otherwise an IllegalArgumentException is thrown.</p>
-   *
-   * @see #setMergeFactor
-   * @deprecated use {@link LogMergePolicy#getMergeFactor()} directly.
-   */
-  @Deprecated
-  public int getMergeFactor() {
-    return getLogMergePolicy().getMergeFactor();
-  }
-
   /** If non-null, this will be the default infoStream used
    * by a newly instantiated IndexWriter.
    * @see #setInfoStream
@@ -1596,52 +914,6 @@ public class IndexWriter implements Clos
   }
   
   /**
-   * Sets the maximum time to wait for a write lock (in milliseconds) for this instance of IndexWriter.  @see
-   * @see #setDefaultWriteLockTimeout to change the default value for all instances of IndexWriter.
-   * @deprecated use {@link IndexWriterConfig#setWriteLockTimeout(long)} instead
-   */
-  @Deprecated
-  public void setWriteLockTimeout(long writeLockTimeout) {
-    ensureOpen();
-    this.writeLockTimeout = writeLockTimeout;
-    // Required so config.getWriteLockTimeout returns the right value. But this
-    // will go away together with the method in 4.0.
-    config.setWriteLockTimeout(writeLockTimeout);
-  }
-
-  /**
-   * Returns allowed timeout when acquiring the write lock.
-   * @see #setWriteLockTimeout
-   * @deprecated use {@link IndexWriterConfig#getWriteLockTimeout()}
-   */
-  @Deprecated
-  public long getWriteLockTimeout() {
-    ensureOpen();
-    return writeLockTimeout;
-  }
-
-  /**
-   * Sets the default (for any instance of IndexWriter) maximum time to wait for a write lock (in
-   * milliseconds).
-   * @deprecated use {@link IndexWriterConfig#setDefaultWriteLockTimeout(long)} instead
-   */
-  @Deprecated
-  public static void setDefaultWriteLockTimeout(long writeLockTimeout) {
-    IndexWriterConfig.setDefaultWriteLockTimeout(writeLockTimeout);
-  }
-
-  /**
-   * Returns default write lock timeout for newly
-   * instantiated IndexWriters.
-   * @see #setDefaultWriteLockTimeout
-   * @deprecated use {@link IndexWriterConfig#getDefaultWriteLockTimeout()} instead
-   */
-  @Deprecated
-  public static long getDefaultWriteLockTimeout() {
-    return IndexWriterConfig.getDefaultWriteLockTimeout();
-  }
-
-  /**
    * Commits all changes to an index and closes all
    * associated files.  Note that this may be a costly
    * operation, so, try to re-use a single writer instead of
@@ -3030,7 +2302,7 @@ public class IndexWriter implements Clos
       }
       
       // Now create the compound file if needed
-      if (mergePolicy instanceof LogMergePolicy && getUseCompoundFile()) {
+      if (mergePolicy instanceof LogMergePolicy && ((LogMergePolicy) mergePolicy).getUseCompoundFile()) {
 
         List<String> files = null;
 
@@ -3998,7 +3270,7 @@ public class IndexWriter implements Clos
     }
   }
 
-  private final synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
+  private synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
     final int numSegments = merge.segments.size();
     if (suppressExceptions) {
       // Suppress any new exceptions so we throw the
@@ -4043,7 +3315,7 @@ public class IndexWriter implements Clos
   /** Does the actual (time-consuming) work of the merge,
    *  but without holding synchronized lock on IndexWriter
    *  instance */
-  final private int mergeMiddle(MergePolicy.OneMerge merge) 
+  private int mergeMiddle(MergePolicy.OneMerge merge)
     throws CorruptIndexException, IOException {
     
     merge.checkAborted(directory);
@@ -4507,63 +3779,6 @@ public class IndexWriter implements Clos
     directory.makeLock(IndexWriter.WRITE_LOCK_NAME).release();
   }
 
-  /**
-   * Specifies maximum field length (in number of tokens/terms) in
-   * {@link IndexWriter} constructors. {@link #setMaxFieldLength(int)} overrides
-   * the value set by the constructor.
-   * 
-   * @deprecated use {@link IndexWriterConfig} and pass
-   *             {@link IndexWriterConfig#UNLIMITED_FIELD_LENGTH} or your own
-   *             value.
-   */
-  @Deprecated
-  public static final class MaxFieldLength {
-
-    private int limit;
-    private String name;
-
-    /**
-     * Private type-safe-enum-pattern constructor.
-     * 
-     * @param name instance name
-     * @param limit maximum field length
-     */
-    private MaxFieldLength(String name, int limit) {
-      this.name = name;
-      this.limit = limit;
-    }
-
-    /**
-     * Public constructor to allow users to specify the maximum field size limit.
-     * 
-     * @param limit The maximum field length
-     */
-    public MaxFieldLength(int limit) {
-      this("User-specified", limit);
-    }
-    
-    public int getLimit() {
-      return limit;
-    }
-    
-    @Override
-    public String toString()
-    {
-      return name + ":" + limit;
-    }
-
-    /** Sets the maximum field length to {@link Integer#MAX_VALUE}. */
-    public static final MaxFieldLength UNLIMITED
-        = new MaxFieldLength("UNLIMITED", Integer.MAX_VALUE);
-
-    /**
-     *  Sets the maximum field length to 
-     * {@link #DEFAULT_MAX_FIELD_LENGTH} 
-     * */
-    public static final MaxFieldLength LIMITED
-        = new MaxFieldLength("LIMITED", DEFAULT_MAX_FIELD_LENGTH);
-  }
-
   /** If {@link #getReader} has been called (ie, this writer
    *  is in near real-time mode), then after a merge
    *  completes, this class can be invoked to warm the
@@ -4582,31 +3797,6 @@ public class IndexWriter implements Clos
 
   private IndexReaderWarmer mergedSegmentWarmer;
 
-  /**
-   * Set the merged segment warmer. See {@link IndexReaderWarmer}.
-   * 
-   * @deprecated use
-   *             {@link IndexWriterConfig#setMergedSegmentWarmer}
-   *             instead.
-   */
-  @Deprecated
-  public void setMergedSegmentWarmer(IndexReaderWarmer warmer) {
-    mergedSegmentWarmer = warmer;
-    // Required so config.getMergedSegmentWarmer returns the right value. But
-    // this will go away together with the method in 4.0.
-    config.setMergedSegmentWarmer(mergedSegmentWarmer);
-  }
-
-  /**
-   * Returns the current merged segment warmer. See {@link IndexReaderWarmer}.
-   * 
-   * @deprecated use {@link IndexWriterConfig#getMergedSegmentWarmer()} instead.
-   */
-  @Deprecated
-  public IndexReaderWarmer getMergedSegmentWarmer() {
-    return mergedSegmentWarmer;
-  }
-
   private void handleOOM(OutOfMemoryError oom, String location) {
     if (infoStream != null) {
       message("hit OutOfMemoryError inside " + location);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Tue Nov 30 11:22:39 2010
@@ -1240,33 +1240,6 @@ public class SegmentReader extends Index
   public final Object getCoreCacheKey() {
     return core;
   }
-  
-  /**
-   * Lotsa tests did hacks like:<br/>
-   * SegmentReader reader = (SegmentReader) IndexReader.open(dir);<br/>
-   * They broke. This method serves as a hack to keep hacks working
-   * We do it with R/W access for the tests (BW compatibility)
-   * @deprecated Remove this when tests are fixed!
-   */
-  @Deprecated
-  static SegmentReader getOnlySegmentReader(Directory dir) throws IOException {
-    return getOnlySegmentReader(IndexReader.open(dir, false));
-  }
-
-  static SegmentReader getOnlySegmentReader(IndexReader reader) {
-    if (reader instanceof SegmentReader)
-      return (SegmentReader) reader;
-
-    if (reader instanceof DirectoryReader) {
-      IndexReader[] subReaders = reader.getSequentialSubReaders();
-      if (subReaders.length != 1)
-        throw new IllegalArgumentException(reader + " has " + subReaders.length + " segments instead of exactly one");
-
-      return (SegmentReader) subReaders[0];
-    }
-
-    throw new IllegalArgumentException(reader + " is not a SegmentReader or a single-segment DirectoryReader");
-  }
 
   @Override
   public int getTermInfosIndexDivisor() {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/Term.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/Term.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/Term.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/Term.java Tue Nov 30 11:22:39 2010
@@ -168,7 +168,7 @@ public final class Term implements Compa
     BytesRef.getUTF8SortedAsUTF16Comparator();
 
   /** 
-   * @deprecated For internal backwards compatibility use only
+   * @deprecated (4.0) For internal backwards compatibility use only
    * @lucene.internal
    */
   @Deprecated

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java Tue Nov 30 11:22:39 2010
@@ -32,7 +32,7 @@ import org.apache.lucene.index.codecs.Fi
  *  format.  It does not provide a writer because newly
  *  written segments should use StandardCodec.
  *
- * @deprecated This is only used to read indexes created
+ * @deprecated (4.0) This is only used to read indexes created
  * before 4.0.
  * @lucene.experimental
  */

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Tue Nov 30 11:22:39 2010
@@ -44,7 +44,10 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.UnicodeUtil;
 
 /** Exposes flex API on a pre-flex index, as a codec. 
- * @lucene.experimental */
+ * @lucene.experimental
+ * @deprecated (4.0)
+ */
+@Deprecated
 public class PreFlexFields extends FieldsProducer {
   
   private static final boolean DEBUG_SURROGATES = false;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java Tue Nov 30 11:22:39 2010
@@ -26,7 +26,7 @@ import org.apache.lucene.index.codecs.st
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 
-/** @deprecated 
+/** @deprecated (4.0)
  *  @lucene.experimental */
 @Deprecated
 public class SegmentTermDocs {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java Tue Nov 30 11:22:39 2010
@@ -26,7 +26,7 @@ import org.apache.lucene.index.IndexForm
 import org.apache.lucene.index.IndexFormatTooNewException;
 
 /**
- * @deprecated No longer used with flex indexing, except for
+ * @deprecated (4.0) No longer used with flex indexing, except for
  * reading old segments 
  * @lucene.experimental */
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java Tue Nov 30 11:22:39 2010
@@ -23,7 +23,11 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.IndexInput;
 
-/** @lucene.experimental */
+/**
+ * @lucene.experimental
+ * @deprecated (4.0)
+ */
+@Deprecated
 public final class SegmentTermPositions
 extends SegmentTermDocs  {
   private IndexInput proxStream;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java Tue Nov 30 11:22:39 2010
@@ -25,6 +25,11 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.FieldInfos;
 
+/**
+ * @lucene.experimental
+ * @deprecated (4.0)
+ */
+@Deprecated
 final class TermBuffer implements Cloneable {
 
   private String field;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java Tue Nov 30 11:22:39 2010
@@ -19,7 +19,7 @@ package org.apache.lucene.index.codecs.p
 
 /** A TermInfo is the record of information stored for a
  * term
- * @deprecated This class is no longer used in flexible
+ * @deprecated (4.0) This class is no longer used in flexible
  * indexing. */
 
 @Deprecated

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Tue Nov 30 11:22:39 2010
@@ -30,7 +30,7 @@ import org.apache.lucene.util.DoubleBarr
 /** This stores a monotonically increasing set of <Term, TermInfo> pairs in a
  * Directory.  Pairs are accessed either by Term or by ordinal position the
  * set
- * @deprecated This class has been replaced by
+ * @deprecated (4.0) This class has been replaced by
  * FormatPostingsTermsDictReader, except for reading old segments. 
  * @lucene.experimental
  */

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/CharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/CharStream.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/CharStream.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/CharStream.java Tue Nov 30 11:22:39 2010
@@ -28,14 +28,14 @@ public interface CharStream {
 
   /**
    * Returns the column position of the character last read.
-   * @deprecated
+   * @deprecated (gen)
    * @see #getEndColumn
    */
   int getColumn();
 
   /**
    * Returns the line number of the character last read.
-   * @deprecated
+   * @deprecated (gen)
    * @see #getEndLine
    */
   int getLine();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParser.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParser.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParser.java Tue Nov 30 11:22:39 2010
@@ -18,7 +18,6 @@ import org.apache.lucene.analysis.Cachin
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.DateField;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
@@ -77,10 +76,8 @@ import org.apache.lucene.util.VirtualMet
  * <tt>date:[6/1/2005 TO 6/4/2005]</tt> produces a range query that searches
  * for "date" fields between 2005-06-01 and 2005-06-04. Note that the format
  * of the accepted input depends on {@link #setLocale(Locale) the locale}.
- * By default a date is converted into a search term using the deprecated
- * {@link DateField} for compatibility reasons.
- * To use the new {@link DateTools} to convert dates, a
- * {@link org.apache.lucene.document.DateTools.Resolution} has to be set.
+ * A {@link org.apache.lucene.document.DateTools.Resolution} has to be set,
+ * if you want to use {@link DateTools} for date conversion.
  * </p>
  * <p>
  * The date resolution that shall be used for RangeQueries can be set
@@ -91,10 +88,9 @@ import org.apache.lucene.util.VirtualMet
  * resolutions take, if set, precedence over the default date resolution.
  * </p>
  * <p>
- * If you use neither {@link DateField} nor {@link DateTools} in your
- * index, you can create your own
+ * If you don't use {@link DateTools} in your index, you can create your own
  * query parser that inherits QueryParser and overwrites
- * {@link #getRangeQuery(String, String, String, boolean)} to
+ * {@link #getRangeQuery(String, String, String, boolean, boolean)} to
  * use a different method for date conversion.
  * </p>
  *
@@ -108,8 +104,6 @@ import org.apache.lucene.util.VirtualMet
  * <p><b>NOTE</b>: You must specify the required {@link Version}
  * compatibility when creating QueryParser:
  * <ul>
- *    <li> As of 2.9, {@link #setEnablePositionIncrements} is true by
- *         default.
  *    <li> As of 3.1, {@link #setAutoGeneratePhraseQueries} is false by
  *         default.
  * </ul>

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParser.jj
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParser.jj?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParser.jj (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParser.jj Tue Nov 30 11:22:39 2010
@@ -42,7 +42,6 @@ import org.apache.lucene.analysis.Cachin
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.DateField;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
@@ -101,10 +100,8 @@ import org.apache.lucene.util.VirtualMet
  * <tt>date:[6/1/2005 TO 6/4/2005]</tt> produces a range query that searches
  * for "date" fields between 2005-06-01 and 2005-06-04. Note that the format
  * of the accepted input depends on {@link #setLocale(Locale) the locale}.
- * By default a date is converted into a search term using the deprecated
- * {@link DateField} for compatibility reasons.
- * To use the new {@link DateTools} to convert dates, a
- * {@link org.apache.lucene.document.DateTools.Resolution} has to be set.
+ * A {@link org.apache.lucene.document.DateTools.Resolution} has to be set,
+ * if you want to use {@link DateTools} for date conversion.
  * </p>
  * <p>
  * The date resolution that shall be used for RangeQueries can be set
@@ -115,10 +112,9 @@ import org.apache.lucene.util.VirtualMet
  * resolutions take, if set, precedence over the default date resolution.
  * </p>
  * <p>
- * If you use neither {@link DateField} nor {@link DateTools} in your
- * index, you can create your own
+ * If you don't use {@link DateTools} in your index, you can create your own
  * query parser that inherits QueryParser and overwrites
- * {@link #getRangeQuery(String, String, String, boolean)} to
+ * {@link #getRangeQuery(String, String, String, boolean, boolean)} to
  * use a different method for date conversion.
  * </p>
  *
@@ -132,8 +128,6 @@ import org.apache.lucene.util.VirtualMet
  * <p><b>NOTE</b>: You must specify the required {@link Version}
  * compatibility when creating QueryParser:
  * <ul>
- *    <li> As of 2.9, {@link #setEnablePositionIncrements} is true by
- *         default.
  *    <li> As of 3.1, {@link #setAutoGeneratePhraseQueries} is false by
  *         default.
  * </ul>

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java Tue Nov 30 11:22:39 2010
@@ -17,25 +17,23 @@
 
 package org.apache.lucene.queryParser;
 
+import java.io.IOException;
+import java.io.StringReader;
+import java.text.Collator;
+import java.text.DateFormat;
+import java.util.*;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CachingTokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.DateField;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.queryParser.QueryParser.Operator;
 import org.apache.lucene.search.*;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Version;
-import org.apache.lucene.util.VirtualMethod;
-import org.apache.lucene.queryParser.QueryParser.Operator;
-
-import java.io.IOException;
-import java.io.StringReader;
-import java.text.Collator;
-import java.text.DateFormat;
-import java.util.*;
 
 /** This class is overridden by QueryParser in QueryParser.jj
  * and acts to eparate the majority of the Java code from the .jj grammar file. 
@@ -84,20 +82,6 @@ public abstract class QueryParserBase {
   // for use when constructing RangeQuerys.
   Collator rangeCollator = null;
 
-  /** @deprecated remove when getFieldQuery is removed */
-  @Deprecated
-  static final VirtualMethod<QueryParserBase> getFieldQueryMethod =
-    new VirtualMethod<QueryParserBase>(QueryParserBase.class, "getFieldQuery", String.class, String.class);
-  /** @deprecated remove when getFieldQuery is removed */
-  @Deprecated
-  static final VirtualMethod<QueryParserBase> getFieldQueryWithQuotedMethod =
-    new VirtualMethod<QueryParserBase>(QueryParserBase.class, "getFieldQuery", String.class, String.class, boolean.class);
-  /** @deprecated remove when getFieldQuery is removed */
-  @Deprecated
-  final boolean hasNewAPI =
-    VirtualMethod.compareImplementationDistance(getClass(),
-        getFieldQueryWithQuotedMethod, getFieldQueryMethod) >= 0; // its ok for both to be overridden
-
   boolean autoGeneratePhraseQueries;
 
   // So the generated QueryParser(CharStream) won't error out
@@ -112,11 +96,6 @@ public abstract class QueryParserBase {
   public void init(Version matchVersion, String f, Analyzer a) {
     analyzer = a;
     field = f;
-    if (matchVersion.onOrAfter(Version.LUCENE_29)) {
-      enablePositionIncrements = true;
-    } else {
-      enablePositionIncrements = false;
-    }
     if (matchVersion.onOrAfter(Version.LUCENE_31)) {
       setAutoGeneratePhraseQueries(false);
     } else {
@@ -190,9 +169,6 @@ public abstract class QueryParserBase {
    * surrounded by double quotes.
    */
   public final void setAutoGeneratePhraseQueries(boolean value) {
-    if (value == false && !hasNewAPI)
-      throw new IllegalArgumentException("You must implement the new API: getFieldQuery(String,String,boolean)"
-       + " to use setAutoGeneratePhraseQueries(false)");
     this.autoGeneratePhraseQueries = value;
   }
 
@@ -272,7 +248,7 @@ public abstract class QueryParserBase {
    * Useful when e.g. a StopFilter increases the position increment of
    * the token that follows an omitted token.
    * <p>
-   * Default: false.
+   * Default: true.
    */
   public void setEnablePositionIncrements(boolean enable) {
     this.enablePositionIncrements = enable;
@@ -489,15 +465,6 @@ public abstract class QueryParserBase {
   }
 
   /**
-   * @deprecated Use {@link #getFieldQuery(String,String,boolean)} instead.
-   */
-  @Deprecated
-  protected Query getFieldQuery(String field, String queryText) throws ParseException {
-    // treat the text as if it was quoted, to drive phrase logic with old versions.
-    return getFieldQuery(field, queryText, true);
-  }
-
-  /**
    * @exception org.apache.lucene.queryParser.ParseException throw in overridden method to disallow
    */
   protected Query getFieldQuery(String field, String queryText, boolean quoted)  throws ParseException {
@@ -684,7 +651,7 @@ public abstract class QueryParserBase {
    */
   protected Query getFieldQuery(String field, String queryText, int slop)
         throws ParseException {
-    Query query = hasNewAPI ? getFieldQuery(field, queryText, true) : getFieldQuery(field, queryText);
+    Query query = getFieldQuery(field, queryText, true);
 
     if (query instanceof PhraseQuery) {
       ((PhraseQuery) query).setSlop(slop);
@@ -696,11 +663,6 @@ public abstract class QueryParserBase {
     return query;
   }
 
-
-  @Deprecated
-  protected final Query getRangeQuery(String field, String part1, String part2, boolean inclusive) throws MethodRemovedUseAnother {return null;}
-
-
   /**
    *
    * @exception org.apache.lucene.queryParser.ParseException
@@ -722,15 +684,7 @@ public abstract class QueryParserBase {
     DateTools.Resolution resolution = getDateResolution(field);
     
     try {
-      Date d1 = df.parse(part1);
-      if (resolution == null) {
-        // no default or field specific date resolution has been set,
-        // use deprecated DateField to maintain compatibility with
-        // pre-1.9 Lucene versions.
-        part1 = DateField.dateToString(d1);
-      } else {
-        part1 = DateTools.dateToString(d1, resolution);
-      }
+      part1 = DateTools.dateToString(df.parse(part1), resolution);
     } catch (Exception e) { }
 
     try {
@@ -747,14 +701,7 @@ public abstract class QueryParserBase {
         cal.set(Calendar.MILLISECOND, 999);
         d2 = cal.getTime();
       }
-      if (resolution == null) {
-        // no default or field specific date resolution has been set,
-        // use deprecated DateField to maintain compatibility with
-        // pre-1.9 Lucene versions.
-        part2 = DateField.dateToString(d2);
-      } else {
-        part2 = DateTools.dateToString(d2, resolution);
-      }
+      part2 = DateTools.dateToString(d2, resolution);
     } catch (Exception e) { }
 
     return newRangeQuery(field, part1, part2, startInclusive, endInclusive);
@@ -838,10 +785,6 @@ public abstract class QueryParserBase {
     return new FuzzyQuery(term,minimumSimilarity,prefixLength);
   }
 
-  @Deprecated
-  protected final Query newRangeQuery(String field, String part1, String part2, boolean inclusive) throws MethodRemovedUseAnother {return null;}
-
-
   /**
    * Builds a new TermRangeQuery instance
    * @param field Field
@@ -1064,7 +1007,7 @@ public abstract class QueryParserBase {
       }
       q = getFuzzyQuery(qfield, termImage, fms);
     } else {
-      q = hasNewAPI ? getFieldQuery(qfield, termImage, false) : getFieldQuery(qfield, termImage);
+      q = getFieldQuery(qfield, termImage, false);
     }
     return q;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java Tue Nov 30 11:22:39 2010
@@ -16,7 +16,6 @@ import org.apache.lucene.analysis.Cachin
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.DateField;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java Tue Nov 30 11:22:39 2010
@@ -193,7 +193,7 @@ class BooleanScorer2 extends Scorer {
   }
 
   private Scorer dualConjunctionSumScorer(Scorer req1, Scorer req2) throws IOException { // non counting.
-    return new ConjunctionScorer(defaultSimilarity, new Scorer[]{req1, req2});
+    return new ConjunctionScorer(defaultSimilarity, req1, req2);
     // All scorers match, so defaultSimilarity always has 1 as
     // the coordination factor.
     // Therefore the sum of the scores of two scorers

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Similarity.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Similarity.java Tue Nov 30 11:22:39 2010
@@ -18,15 +18,15 @@ package org.apache.lucene.search;
  */
 
 
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Explanation.IDFExplanation;
 import org.apache.lucene.util.SmallFloat;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-
 
 /** 
  * Expert: Scoring API.
@@ -562,16 +562,6 @@ public abstract class Similarity impleme
       NORM_TABLE[i] = SmallFloat.byte315ToFloat((byte)i);
   }
 
-  /**
-   * Decodes a normalization factor stored in an index.
-   * @see #decodeNormValue(byte)
-   * @deprecated Use {@link #decodeNormValue} instead.
-   */
-  @Deprecated
-  public static float decodeNorm(byte b) {
-    return NORM_TABLE[b & 0xFF];  // & 0xFF maps negative bytes to positive above 127
-  }
-
   /** Decodes a normalization factor stored in an index.
    * @see #encodeNormValue(float)
    */
@@ -579,17 +569,6 @@ public abstract class Similarity impleme
     return NORM_TABLE[b & 0xFF];  // & 0xFF maps negative bytes to positive above 127
   }
 
-  /** Returns a table for decoding normalization bytes.
-   * @see #encodeNormValue(float)
-   * @see #decodeNormValue(byte)
-   * 
-   * @deprecated Use instance methods for encoding/decoding norm values to enable customization.
-   */
-  @Deprecated
-  public static float[] getNormDecoder() {
-    return NORM_TABLE;
-  }
-
   /**
    * Compute the normalization value for a field, given the accumulated
    * state of term processing for this field (see {@link FieldInvertState}).
@@ -670,20 +649,6 @@ public abstract class Similarity impleme
     return SmallFloat.floatToByte315(f);
   }
   
-  /**
-   * Static accessor kept for backwards compability reason, use encodeNormValue instead.
-   * @param f norm-value to encode
-   * @return byte representing the given float
-   * @deprecated Use {@link #encodeNormValue} instead.
-   * 
-   * @see #encodeNormValue(float)
-   */
-  @Deprecated
-  public static byte encodeNorm(float f) {
-    return SmallFloat.floatToByte315(f);
-  }
-
-
   /** Computes a score factor based on a term or phrase's frequency in a
    * document.  This value is multiplied by the {@link #idf(int, int)}
    * factor for each term in the query and these products are then summed to

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SortField.java Tue Nov 30 11:22:39 2010
@@ -21,13 +21,7 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.util.Locale;
 
-import org.apache.lucene.search.cache.ByteValuesCreator;
-import org.apache.lucene.search.cache.CachedArrayCreator;
-import org.apache.lucene.search.cache.DoubleValuesCreator;
-import org.apache.lucene.search.cache.FloatValuesCreator;
-import org.apache.lucene.search.cache.IntValuesCreator;
-import org.apache.lucene.search.cache.LongValuesCreator;
-import org.apache.lucene.search.cache.ShortValuesCreator;
+import org.apache.lucene.search.cache.*;
 import org.apache.lucene.util.StringHelper;
 
 /**
@@ -138,7 +132,7 @@ implements Serializable {
    * @throws IllegalArgumentException if the parser fails to
    *  subclass an existing numeric parser, or field is null
    *  
-   *  @deprecated use EntryCreator version
+   *  @deprecated (4.0) use EntryCreator version
    */
   @Deprecated
   public SortField (String field, FieldCache.Parser parser) {
@@ -156,7 +150,7 @@ implements Serializable {
    * @throws IllegalArgumentException if the parser fails to
    *  subclass an existing numeric parser, or field is null
    *  
-   *  @deprecated use EntryCreator version
+   *  @deprecated (4.0) use EntryCreator version
    */
   @Deprecated
   public SortField (String field, FieldCache.Parser parser, boolean reverse) {
@@ -314,7 +308,7 @@ implements Serializable {
   /** Returns the instance of a {@link FieldCache} parser that fits to the given sort type.
    * May return <code>null</code> if no parser was specified. Sorting is using the default parser then.
    * @return An instance of a {@link FieldCache} parser, or <code>null</code>.
-   * @deprecated use getEntryCreator()
+   * @deprecated (4.0) use getEntryCreator()
    */
   @Deprecated
   public FieldCache.Parser getParser() {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java Tue Nov 30 11:22:39 2010
@@ -20,10 +20,10 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.text.Collator;
 
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.util.ToStringUtils;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.ToStringUtils;
 
 /**
  * A Query that matches documents within an range of terms.
@@ -143,12 +143,6 @@ public class TermRangeQuery extends Mult
         lowerTerm, upperTerm, includeLower, includeUpper, collator);
   }
 
-  /** @deprecated */
-  @Deprecated
-  public String field() {
-    return getField();
-  }
-
   /** Prints a user-readable version of this query. */
   @Override
   public String toString(String field) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java Tue Nov 30 11:22:39 2010
@@ -19,9 +19,9 @@ package org.apache.lucene.search.functio
 
 import java.io.IOException;
 
-import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Explanation;
+import org.apache.lucene.util.ReaderUtil;
 
 /** This class wraps another ValueSource, but protects
  *  against accidental double RAM usage in FieldCache when
@@ -31,7 +31,7 @@ import org.apache.lucene.search.Explanat
  *  lookup, as it must resolve the incoming document to the
  *  right sub-reader using a binary search.</p>
  *
- *  @deprecated This class is temporary, to ease the
+ *  @deprecated (4.0) This class is temporary, to ease the
  *  migration to segment-based searching. Please change your
  *  code to not pass composite readers to these APIs. */
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java Tue Nov 30 11:22:39 2010
@@ -105,59 +105,6 @@ public abstract class DataOutput {
     writeBytes(utf8Result.bytes, 0, utf8Result.length);
   }
 
-  /** Writes a sub sequence of characters from s as the old
-   *  format (modified UTF-8 encoded bytes).
-   * @param s the source of the characters
-   * @param start the first character in the sequence
-   * @param length the number of characters in the sequence
-   * @deprecated -- please pre-convert to utf8 bytes
-   * instead or use {@link #writeString}
-   */
-  @Deprecated
-  public void writeChars(String s, int start, int length)
-       throws IOException {
-    final int end = start + length;
-    for (int i = start; i < end; i++) {
-      final int code = s.charAt(i);
-      if (code >= 0x01 && code <= 0x7F)
-        writeByte((byte)code);
-      else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
-        writeByte((byte)(0xC0 | (code >> 6)));
-        writeByte((byte)(0x80 | (code & 0x3F)));
-      } else {
-        writeByte((byte)(0xE0 | (code >>> 12)));
-        writeByte((byte)(0x80 | ((code >> 6) & 0x3F)));
-        writeByte((byte)(0x80 | (code & 0x3F)));
-      }
-    }
-  }
-
-  /** Writes a sub sequence of characters from char[] as
-   *  the old format (modified UTF-8 encoded bytes).
-   * @param s the source of the characters
-   * @param start the first character in the sequence
-   * @param length the number of characters in the sequence
-   * @deprecated -- please pre-convert to utf8 bytes instead or use {@link #writeString}
-   */
-  @Deprecated
-  public void writeChars(char[] s, int start, int length)
-    throws IOException {
-    final int end = start + length;
-    for (int i = start; i < end; i++) {
-      final int code = s[i];
-      if (code >= 0x01 && code <= 0x7F)
-        writeByte((byte)code);
-      else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
-        writeByte((byte)(0xC0 | (code >> 6)));
-        writeByte((byte)(0x80 | (code & 0x3F)));
-      } else {
-        writeByte((byte)(0xE0 | (code >>> 12)));
-        writeByte((byte)(0x80 | ((code >> 6) & 0x3F)));
-        writeByte((byte)(0x80 | (code & 0x3F)));
-      }
-    }
-  }
-
   private static int COPY_BUFFER_SIZE = 16384;
   private byte[] copyBuffer;
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/Directory.java Tue Nov 30 11:22:39 2010
@@ -95,19 +95,6 @@ public abstract class Directory implemen
        throws IOException;
 
   /**
-   * Ensure that any writes to this file are moved to
-   * stable storage.  Lucene uses this to properly commit
-   * changes to the index, to prevent a machine/OS crash
-   * from corrupting the index.
-   * @deprecated use {@link #sync(Collection)} instead.
-   * For easy migration you can change your code to call
-   * sync(Collections.singleton(name))
-   */
-  @Deprecated
-  public void sync(String name) throws IOException { // TODO 4.0 kill me
-  }
-
-  /**
    * Ensure that any writes to these files are moved to
    * stable storage.  Lucene uses this to properly commit
    * changes to the index, to prevent a machine/OS crash
@@ -118,10 +105,7 @@ public abstract class Directory implemen
    * For other impls the operation can be a noop, for various
    * reasons.
    */
-  public void sync(Collection<String> names) throws IOException { // TODO 4.0 make me abstract
-    for (String name : names)
-      sync(name);
-  }
+  public abstract void sync(Collection<String> names) throws IOException;
 
   /** Returns a stream reading an existing file. */
   public abstract IndexInput openInput(String name)
@@ -233,41 +217,6 @@ public abstract class Directory implemen
   }
 
   /**
-   * Copy contents of a directory src to a directory dest. If a file in src
-   * already exists in dest then the one in dest will be blindly overwritten.
-   * <p>
-   * <b>NOTE:</b> the source directory cannot change while this method is
-   * running. Otherwise the results are undefined and you could easily hit a
-   * FileNotFoundException.
-   * <p>
-   * <b>NOTE:</b> this method only copies files that look like index files (ie,
-   * have extensions matching the known extensions of index files).
-   * 
-   * @param src source directory
-   * @param dest destination directory
-   * @param closeDirSrc if <code>true</code>, call {@link #close()} method on 
-   *        source directory
-   * @deprecated should be replaced with calls to
-   *             {@link #copy(Directory, String, String)} for every file that
-   *             needs copying. You can use the following code:
-   * 
-   * <pre>
-   * for (String file : src.listAll()) {
-   *   src.copy(dest, file, file);
-   * }
-   * </pre>
-   */
-  @Deprecated
-  public static void copy(Directory src, Directory dest, boolean closeDirSrc) throws IOException {
-    for (String file : src.listAll()) {
-      src.copy(dest, file, file);
-    }
-    if (closeDirSrc) {
-      src.close();
-    }
-  }
-
-  /**
    * @throws AlreadyClosedException if this Directory is closed
    */
   protected final void ensureOpen() throws AlreadyClosedException {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FSDirectory.java Tue Nov 30 11:22:39 2010
@@ -321,12 +321,6 @@ public abstract class FSDirectory extend
     staleFiles.add(io.name);
   }
 
-  @Deprecated
-  @Override
-  public void sync(String name) throws IOException {
-    sync(Collections.singleton(name));
-  }
-
   @Override
   public void sync(Collection<String> names) throws IOException {
     ensureOpen();
@@ -383,12 +377,6 @@ public abstract class FSDirectory extend
     isOpen = false;
   }
 
-  /** @deprecated Use {@link #getDirectory} instead. */
-  @Deprecated
-  public File getFile() {
-    return getDirectory();
-  }
-
   /** @return the underlying filesystem directory */
   public File getDirectory() {
     ensureOpen();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java Tue Nov 30 11:22:39 2010
@@ -135,12 +135,6 @@ public class FileSwitchDirectory extends
     return getDirectory(name).createOutput(name);
   }
 
-  @Deprecated
-  @Override
-  public void sync(String name) throws IOException {
-    sync(Collections.singleton(name));
-  }
-
   @Override
   public void sync(Collection<String> names) throws IOException {
     List<String> primaryNames = new ArrayList<String>();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NoLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NoLockFactory.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NoLockFactory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/NoLockFactory.java Tue Nov 30 11:22:39 2010
@@ -33,14 +33,7 @@ public class NoLockFactory extends LockF
   private static NoLock singletonLock = new NoLock();
   private static NoLockFactory singleton = new NoLockFactory();
   
-  /**
-   * @deprecated This constructor was not intended to be public and should not be used.
-   *  It will be made private in Lucene 4.0
-   * @see #getNoLockFactory()
-   */
-  // make private in 4.0!
-  @Deprecated
-  public NoLockFactory() {}
+  private NoLockFactory() {}
 
   public static NoLockFactory getNoLockFactory() {
     return singleton;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1040463&r1=1040462&r2=1040463&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMDirectory.java Tue Nov 30 11:22:39 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.store;
 import java.io.IOException;
 import java.io.FileNotFoundException;
 import java.io.Serializable;
+import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
@@ -187,6 +188,9 @@ public class RAMDirectory extends Direct
     return new RAMFile(this);
   }
 
+  public void sync(Collection<String> names) throws IOException {
+  }
+
   /** Returns a stream reading an existing file. */
   @Override
   public IndexInput openInput(String name) throws IOException {