mirror of https://github.com/apache/lucene.git
LUCENE-4055: remove dead code and unnecessary sync
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1340151 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a9721702e1
commit
b0ec623d46
|
@ -414,7 +414,7 @@ class DocumentsWriterPerThread {
|
|||
private void doAfterFlush() throws IOException {
|
||||
segment = null;
|
||||
consumer.doAfterFlush();
|
||||
fieldInfos = FieldInfos.Builder.from(fieldInfos);
|
||||
fieldInfos = new FieldInfos.Builder(fieldInfos.globalFieldNumbers);
|
||||
parent.subtractFlushedNumDocs(numDocsInRAM);
|
||||
numDocsInRAM = 0;
|
||||
}
|
||||
|
|
|
@ -199,22 +199,13 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
}
|
||||
|
||||
static final class Builder {
|
||||
private final SortedMap<Integer,FieldInfo> byNumber = new TreeMap<Integer,FieldInfo>();
|
||||
private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
|
||||
private final FieldNumberBiMap globalFieldNumbers;
|
||||
|
||||
private long version; // internal use to track changes
|
||||
final FieldNumberBiMap globalFieldNumbers;
|
||||
|
||||
public Builder() {
|
||||
Builder() {
|
||||
this(new FieldNumberBiMap());
|
||||
}
|
||||
|
||||
public void add(FieldInfos other) {
|
||||
for(FieldInfo fieldInfo : other){
|
||||
add(fieldInfo);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new instance with the given {@link FieldNumberBiMap}.
|
||||
*/
|
||||
|
@ -222,29 +213,24 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
assert globalFieldNumbers != null;
|
||||
this.globalFieldNumbers = globalFieldNumbers;
|
||||
}
|
||||
|
||||
|
||||
public void add(FieldInfos other) {
|
||||
for(FieldInfo fieldInfo : other){
|
||||
add(fieldInfo);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* adds the given field to this FieldInfos name / number mapping. The given FI
|
||||
* must be present in the global field number mapping before this method it
|
||||
* called
|
||||
*/
|
||||
private void putInternal(FieldInfo fi) {
|
||||
assert !byNumber.containsKey(fi.number);
|
||||
assert !byName.containsKey(fi.name);
|
||||
assert globalFieldNumbers == null || globalFieldNumbers.containsConsistent(Integer.valueOf(fi.number), fi.name);
|
||||
byNumber.put(fi.number, fi);
|
||||
assert globalFieldNumbers.containsConsistent(Integer.valueOf(fi.number), fi.name);
|
||||
byName.put(fi.name, fi);
|
||||
}
|
||||
|
||||
private int nextFieldNumber(String name, int preferredFieldNumber) {
|
||||
// get a global number for this field
|
||||
final int fieldNumber = globalFieldNumbers.addOrGet(name,
|
||||
preferredFieldNumber);
|
||||
assert byNumber.get(fieldNumber) == null : "field number " + fieldNumber
|
||||
+ " already taken";
|
||||
return fieldNumber;
|
||||
}
|
||||
|
||||
/** If the field is not yet known, adds it. If it is known, checks to make
|
||||
* sure that the isIndexed flag is the same as was given previously for this
|
||||
* field. If not - marks it as being indexed. Same goes for the TermVector
|
||||
|
@ -258,7 +244,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
* @param indexOptions if term freqs should be omitted for this field
|
||||
*/
|
||||
// nocommit: fix testCodecs to do this another way, its the only user of this
|
||||
synchronized FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
|
||||
FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) {
|
||||
return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType);
|
||||
}
|
||||
|
@ -279,15 +265,13 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
fieldType.indexOptions(), null, null);
|
||||
}
|
||||
|
||||
synchronized private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
|
||||
private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) {
|
||||
if (globalFieldNumbers == null) {
|
||||
throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos");
|
||||
}
|
||||
FieldInfo fi = fieldInfo(name);
|
||||
if (fi == null) {
|
||||
final int fieldNumber = nextFieldNumber(name, preferredFieldNumber);
|
||||
// get a global number for this field
|
||||
final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber);
|
||||
fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType);
|
||||
} else {
|
||||
fi.update(isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions);
|
||||
|
@ -298,11 +282,10 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
fi.setNormValueType(normType);
|
||||
}
|
||||
}
|
||||
version++;
|
||||
return fi;
|
||||
}
|
||||
|
||||
synchronized public FieldInfo add(FieldInfo fi) {
|
||||
public FieldInfo add(FieldInfo fi) {
|
||||
// IMPORTANT - reuse the field number if possible for consistent field numbers across segments
|
||||
return addOrUpdateInternal(fi.name, fi.number, fi.isIndexed(), fi.hasVectors(),
|
||||
fi.omitsNorms(), fi.hasPayloads(),
|
||||
|
@ -312,10 +295,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector, boolean omitNorms, boolean storePayloads,
|
||||
IndexOptions indexOptions, DocValues.Type docValuesType, DocValues.Type normType) {
|
||||
// don't check modifiable here since we use that to initially build up FIs
|
||||
if (globalFieldNumbers != null) {
|
||||
globalFieldNumbers.setIfNotSet(fieldNumber, name);
|
||||
}
|
||||
globalFieldNumbers.setIfNotSet(fieldNumber, name);
|
||||
final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValuesType, normType);
|
||||
putInternal(fi);
|
||||
return fi;
|
||||
|
@ -324,32 +304,10 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
public FieldInfo fieldInfo(String fieldName) {
|
||||
return byName.get(fieldName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the fieldinfo object referenced by the fieldNumber.
|
||||
* @param fieldNumber
|
||||
* @return the FieldInfo object or null when the given fieldNumber
|
||||
* doesn't exist.
|
||||
*/
|
||||
public FieldInfo fieldInfo(int fieldNumber) {
|
||||
return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null;
|
||||
}
|
||||
|
||||
synchronized final long getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
final FieldInfos finish() {
|
||||
// nocommit: bogus we don't clone each FI
|
||||
return new FieldInfos(byNumber.values().toArray(new FieldInfo[byNumber.size()]));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new instance from the given instance.
|
||||
*/
|
||||
// nocommit
|
||||
static Builder from(Builder other) {
|
||||
return new Builder(other.globalFieldNumbers);
|
||||
return new FieldInfos(byName.values().toArray(new FieldInfo[byName.size()]));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue