add --shared
[pylucene.git] / lucene-java-3.4.0 / lucene / src / java / org / apache / lucene / index / BufferedDeletes.java
1 package org.apache.lucene.index;
2
3 /**
4  * Licensed to the Apache Software Foundation (ASF) under one or more
5  * contributor license agreements.  See the NOTICE file distributed with
6  * this work for additional information regarding copyright ownership.
7  * The ASF licenses this file to You under the Apache License, Version 2.0
8  * (the "License"); you may not use this file except in compliance with
9  * the License.  You may obtain a copy of the License at
10  *
11  *     http://www.apache.org/licenses/LICENSE-2.0
12  *
13  * Unless required by applicable law or agreed to in writing, software
14  * distributed under the License is distributed on an "AS IS" BASIS,
15  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16  * See the License for the specific language governing permissions and
17  * limitations under the License.
18  */
19
20 import java.util.ArrayList;
21 import java.util.HashMap;
22 import java.util.List;
23 import java.util.Map;
24 import java.util.concurrent.atomic.AtomicLong;
25 import java.util.concurrent.atomic.AtomicInteger;
26
27 import org.apache.lucene.search.Query;
28 import org.apache.lucene.util.RamUsageEstimator;
29
30 /* Holds buffered deletes, by docID, term or query for a
31  * single segment. This is used to hold buffered pending
32  * deletes against the to-be-flushed segment.  Once the
33  * deletes are pushed (on flush in DocumentsWriter), these
34  * deletes are converted to a FrozenDeletes instance. */
35
36 // NOTE: we are sync'd by BufferedDeletes, ie, all access to
37 // instances of this class is via sync'd methods on
38 // BufferedDeletes
39 class BufferedDeletes {
40
41   /* Rough logic: HashMap has an array[Entry] w/ varying
42      load factor (say 2 * POINTER).  Entry is object w/ Term
43      key, Integer val, int hash, Entry next
44      (OBJ_HEADER + 3*POINTER + INT).  Term is object w/
45      String field and String text (OBJ_HEADER + 2*POINTER).
46      We don't count Term's field since it's interned.
47      Term's text is String (OBJ_HEADER + 4*INT + POINTER +
48      OBJ_HEADER + string.length*CHAR).  Integer is
49      OBJ_HEADER + INT. */
50   final static int BYTES_PER_DEL_TERM = 8*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 5*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 6*RamUsageEstimator.NUM_BYTES_INT;
51
52   /* Rough logic: del docIDs are List<Integer>.  Say list
53      allocates ~2X size (2*POINTER).  Integer is OBJ_HEADER
54      + int */
55   final static int BYTES_PER_DEL_DOCID = 2*RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_INT;
56
57   /* Rough logic: HashMap has an array[Entry] w/ varying
58      load factor (say 2 * POINTER).  Entry is object w/
59      Query key, Integer val, int hash, Entry next
60      (OBJ_HEADER + 3*POINTER + INT).  Query we often
61      undercount (say 24 bytes).  Integer is OBJ_HEADER + INT. */
62   final static int BYTES_PER_DEL_QUERY = 5*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 2*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2*RamUsageEstimator.NUM_BYTES_INT + 24;
63
64   final AtomicInteger numTermDeletes = new AtomicInteger();
65   final Map<Term,Integer> terms = new HashMap<Term,Integer>();
66   final Map<Query,Integer> queries = new HashMap<Query,Integer>();
67   final List<Integer> docIDs = new ArrayList<Integer>();
68
69   public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE);
70
71   final AtomicLong bytesUsed = new AtomicLong();
72
73   private final static boolean VERBOSE_DELETES = false;
74
75   long gen;
76
77   @Override
78   public String toString() {
79     if (VERBOSE_DELETES) {
80       return "gen=" + gen + " numTerms=" + numTermDeletes + ", terms=" + terms
81         + ", queries=" + queries + ", docIDs=" + docIDs + ", bytesUsed="
82         + bytesUsed;
83     } else {
84       String s = "gen=" + gen;
85       if (numTermDeletes.get() != 0) {
86         s += " " + numTermDeletes.get() + " deleted terms (unique count=" + terms.size() + ")";
87       }
88       if (queries.size() != 0) {
89         s += " " + queries.size() + " deleted queries";
90       }
91       if (docIDs.size() != 0) {
92         s += " " + docIDs.size() + " deleted docIDs";
93       }
94       if (bytesUsed.get() != 0) {
95         s += " bytesUsed=" + bytesUsed.get();
96       }
97
98       return s;
99     }
100   }
101   
102   public void addQuery(Query query, int docIDUpto) {
103     Integer current = queries.put(query, docIDUpto);
104     // increment bytes used only if the query wasn't added so far.
105     if (current == null) {
106       bytesUsed.addAndGet(BYTES_PER_DEL_QUERY);
107     }
108   }
109
110   public void addDocID(int docID) {
111     docIDs.add(Integer.valueOf(docID));
112     bytesUsed.addAndGet(BYTES_PER_DEL_DOCID);
113   }
114
115   public void addTerm(Term term, int docIDUpto) {
116     Integer current = terms.get(term);
117     if (current != null && docIDUpto < current) {
118       // Only record the new number if it's greater than the
119       // current one.  This is important because if multiple
120       // threads are replacing the same doc at nearly the
121       // same time, it's possible that one thread that got a
122       // higher docID is scheduled before the other
123       // threads.  If we blindly replace then we can
124       // incorrectly get both docs indexed.
125       return;
126     }
127
128     terms.put(term, Integer.valueOf(docIDUpto));
129     numTermDeletes.incrementAndGet();
130     if (current == null) {
131       bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.text.length() * RamUsageEstimator.NUM_BYTES_CHAR);
132     }
133   }
134  
135   void clear() {
136     terms.clear();
137     queries.clear();
138     docIDs.clear();
139     numTermDeletes.set(0);
140     bytesUsed.set(0);
141   }
142   
143   void clearDocIDs() {
144     bytesUsed.addAndGet(-docIDs.size()*BYTES_PER_DEL_DOCID);
145     docIDs.clear();
146   }
147   
148   boolean any() {
149     return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0;
150   }
151 }