mirror of https://github.com/apache/lucene.git
LUCENE-4055: impl normGen as SI attribute
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1342277 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f7210b2481
commit
e612729e6a
|
@ -23,7 +23,6 @@ import java.util.HashMap;
|
|||
import java.util.IdentityHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
|
@ -71,14 +70,13 @@ class Lucene3xNormsProducer extends PerDocProducer {
|
|||
Directory separateNormsDir = info.dir; // separate norms are never inside CFS
|
||||
maxdoc = info.getDocCount();
|
||||
String segmentName = info.name;
|
||||
Map<Integer,Long> normGen = info.getNormGen();
|
||||
boolean success = false;
|
||||
try {
|
||||
long nextNormSeek = NORMS_HEADER.length; //skip header (header unused for now)
|
||||
for (FieldInfo fi : fields) {
|
||||
if (fi.hasNorms()) {
|
||||
String fileName = getNormFilename(segmentName, normGen, fi.number);
|
||||
Directory d = hasSeparateNorms(normGen, fi.number) ? separateNormsDir : dir;
|
||||
String fileName = getNormFilename(info, fi.number);
|
||||
Directory d = hasSeparateNorms(info, fi.number) ? separateNormsDir : dir;
|
||||
|
||||
// singleNormFile means multiple norms share this file
|
||||
boolean singleNormFile = IndexFileNames.matchesExtension(fileName, NORMS_EXTENSION);
|
||||
|
@ -142,22 +140,24 @@ class Lucene3xNormsProducer extends PerDocProducer {
|
|||
}
|
||||
}
|
||||
|
||||
private static String getNormFilename(String segmentName, Map<Integer,Long> normGen, int number) {
|
||||
if (hasSeparateNorms(normGen, number)) {
|
||||
return IndexFileNames.fileNameFromGeneration(segmentName, SEPARATE_NORMS_EXTENSION + number, normGen.get(number));
|
||||
private static String getNormFilename(SegmentInfo info, int number) {
|
||||
if (hasSeparateNorms(info, number)) {
|
||||
long gen = Long.parseLong(info.getAttribute(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + number));
|
||||
return IndexFileNames.fileNameFromGeneration(info.name, SEPARATE_NORMS_EXTENSION + number, gen);
|
||||
} else {
|
||||
// single file for all norms
|
||||
return IndexFileNames.segmentFileName(segmentName, "", NORMS_EXTENSION);
|
||||
return IndexFileNames.segmentFileName(info.name, "", NORMS_EXTENSION);
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean hasSeparateNorms(Map<Integer,Long> normGen, int number) {
|
||||
if (normGen == null) {
|
||||
private static boolean hasSeparateNorms(SegmentInfo info, int number) {
|
||||
String v = info.getAttribute(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + number);
|
||||
if (v == null) {
|
||||
return false;
|
||||
} else {
|
||||
assert Long.parseLong(v) != SegmentInfo.NO;
|
||||
return true;
|
||||
}
|
||||
|
||||
Long gen = normGen.get(number);
|
||||
return gen != null && gen.longValue() != SegmentInfo.NO;
|
||||
}
|
||||
|
||||
static final class NormSource extends Source {
|
||||
|
|
|
@ -63,6 +63,8 @@ public class Lucene3xSegmentInfoFormat extends SegmentInfoFormat {
|
|||
public static final String DS_OFFSET_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dsoffset";
|
||||
public static final String DS_NAME_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dsname";
|
||||
public static final String DS_COMPOUND_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".dscompound";
|
||||
public static final String NORMGEN_KEY = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".normgen";
|
||||
public static final String NORMGEN_PREFIX = Lucene3xSegmentInfoFormat.class.getSimpleName() + ".normfield";
|
||||
|
||||
/**
|
||||
* @return if this segment shares stored fields & vectors, this
|
||||
|
|
|
@ -222,12 +222,15 @@ public class Lucene3xSegmentInfoReader extends SegmentInfoReader {
|
|||
addIfExists(dir, files, IndexFileNames.segmentFileName(name, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION));
|
||||
}
|
||||
|
||||
// parse the normgen stuff and shove it into attributes
|
||||
if (normGen != null) {
|
||||
attributes.put(Lucene3xSegmentInfoFormat.NORMGEN_KEY, Integer.toString(numNormGen));
|
||||
for(Map.Entry<Integer,Long> ent : normGen.entrySet()) {
|
||||
long gen = ent.getValue();
|
||||
if (gen >= SegmentInfo.YES) {
|
||||
// Definitely a separate norm file, with generation:
|
||||
files.add(IndexFileNames.fileNameFromGeneration(name, "s" + ent.getKey(), gen));
|
||||
attributes.put(Lucene3xSegmentInfoFormat.NORMGEN_PREFIX + ent.getKey(), Long.toString(gen));
|
||||
} else if (gen == SegmentInfo.NO) {
|
||||
// No separate norm
|
||||
} else {
|
||||
|
@ -237,8 +240,7 @@ public class Lucene3xSegmentInfoReader extends SegmentInfoReader {
|
|||
}
|
||||
}
|
||||
|
||||
// nocommit: convert normgen into attributes?
|
||||
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, normGen, isCompoundFile,
|
||||
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile,
|
||||
null, diagnostics, Collections.unmodifiableMap(attributes));
|
||||
info.setFiles(files);
|
||||
|
||||
|
@ -256,24 +258,13 @@ public class Lucene3xSegmentInfoReader extends SegmentInfoReader {
|
|||
|
||||
final Map<String,String> attributes = input.readStringStringMap();
|
||||
|
||||
final int numNormGen = input.readInt();
|
||||
final Map<Integer,Long> normGen;
|
||||
if (numNormGen == SegmentInfo.NO) {
|
||||
normGen = null;
|
||||
} else {
|
||||
normGen = new HashMap<Integer, Long>();
|
||||
for(int j=0;j<numNormGen;j++) {
|
||||
normGen.put(j, input.readLong());
|
||||
}
|
||||
}
|
||||
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
|
||||
|
||||
final Map<String,String> diagnostics = input.readStringStringMap();
|
||||
|
||||
final Set<String> files = input.readStringSet();
|
||||
|
||||
// nocommit: convert normgen into attributes?
|
||||
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, normGen, isCompoundFile,
|
||||
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile,
|
||||
null, diagnostics, Collections.unmodifiableMap(attributes));
|
||||
info.setFiles(files);
|
||||
return info;
|
||||
|
|
|
@ -50,13 +50,12 @@ public class Lucene40SegmentInfoReader extends SegmentInfoReader {
|
|||
Lucene40SegmentInfoFormat.VERSION_CURRENT);
|
||||
final String version = input.readString();
|
||||
final int docCount = input.readInt();
|
||||
final Map<Integer,Long> normGen = null;
|
||||
final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
|
||||
final Map<String,String> diagnostics = input.readStringStringMap();
|
||||
final Map<String,String> attributes = input.readStringStringMap();
|
||||
final Set<String> files = input.readStringSet();
|
||||
|
||||
final SegmentInfo si = new SegmentInfo(dir, version, segment, docCount, normGen, isCompoundFile,
|
||||
final SegmentInfo si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile,
|
||||
null, diagnostics, Collections.unmodifiableMap(attributes));
|
||||
si.setFiles(files);
|
||||
|
||||
|
|
|
@ -52,8 +52,6 @@ public class Lucene40SegmentInfoWriter extends SegmentInfoWriter {
|
|||
output.writeString(si.getVersion());
|
||||
output.writeInt(si.getDocCount());
|
||||
|
||||
assert si.getNormGen() == null;
|
||||
|
||||
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
|
||||
output.writeStringStringMap(si.getDiagnostics());
|
||||
output.writeStringStringMap(si.attributes());
|
||||
|
|
|
@ -108,7 +108,7 @@ public class SimpleTextSegmentInfoReader extends SegmentInfoReader {
|
|||
}
|
||||
|
||||
SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount,
|
||||
null, isCompoundFile, null, diagnostics, Collections.unmodifiableMap(attributes));
|
||||
isCompoundFile, null, diagnostics, Collections.unmodifiableMap(attributes));
|
||||
info.setFiles(files);
|
||||
success = true;
|
||||
return info;
|
||||
|
|
|
@ -271,7 +271,7 @@ class DocumentsWriterPerThread {
|
|||
private void initSegmentInfo() {
|
||||
String segment = writer.newSegmentName();
|
||||
segmentInfo = new SegmentInfo(directoryOrig, Constants.LUCENE_MAIN_VERSION, segment, -1,
|
||||
null, false, codec, null, null);
|
||||
false, codec, null, null);
|
||||
assert numDocsInRAM == 0;
|
||||
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
|
||||
|
|
|
@ -2285,7 +2285,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
|
||||
|
||||
SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, -1,
|
||||
null, false, codec, null, null);
|
||||
false, codec, null, null);
|
||||
|
||||
SegmentMerger merger = new SegmentMerger(info, infoStream, trackingDir, config.getTermIndexInterval(),
|
||||
MergeState.CheckAbort.NONE, payloadProcessorProvider,
|
||||
|
@ -2402,7 +2402,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// Same SI as before but we change directory, name and docStoreSegment:
|
||||
SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
|
||||
info.info.getNormGen(), info.info.getUseCompoundFile(),
|
||||
info.info.getUseCompoundFile(),
|
||||
info.info.getCodec(), info.info.getDiagnostics(), Collections.unmodifiableMap(attributes));
|
||||
SegmentInfoPerCommit newInfoPerCommit = new SegmentInfoPerCommit(newInfo, info.getDelCount(), info.getDelGen());
|
||||
|
||||
|
@ -3323,7 +3323,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// ConcurrentMergePolicy we keep deterministic segment
|
||||
// names.
|
||||
final String mergeSegmentName = newSegmentName();
|
||||
SegmentInfo si = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergeSegmentName, -1, null, false, codec, details, null);
|
||||
SegmentInfo si = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergeSegmentName, -1, false, codec, details, null);
|
||||
merge.info = new SegmentInfoPerCommit(si, 0, -1L);
|
||||
|
||||
// Lock order: IW -> BD
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.lucene3x.Lucene3xSegmentInfoFormat;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.TrackingDirectoryWrapper;
|
||||
|
||||
|
@ -45,14 +46,6 @@ public final class SegmentInfo {
|
|||
private int docCount; // number of docs in seg
|
||||
public final Directory dir; // where segment resides
|
||||
|
||||
/*
|
||||
* Current generation of each field's norm file. If this array is null,
|
||||
* means no separate norms. If this array is not null, its values mean:
|
||||
* - NO says this field has no separate norms
|
||||
* >= YES says this field has separate norms with the specified generation
|
||||
*/
|
||||
private final Map<Integer,Long> normGen;
|
||||
|
||||
private boolean isCompoundFile;
|
||||
|
||||
private volatile long sizeInBytes = -1; // total byte size of all files (computed on demand)
|
||||
|
@ -84,14 +77,12 @@ public final class SegmentInfo {
|
|||
* the codecs package.</p>
|
||||
*/
|
||||
public SegmentInfo(Directory dir, String version, String name, int docCount,
|
||||
Map<Integer,Long> normGen, boolean isCompoundFile,
|
||||
Codec codec, Map<String,String> diagnostics, Map<String,String> attributes) {
|
||||
boolean isCompoundFile, Codec codec, Map<String,String> diagnostics, Map<String,String> attributes) {
|
||||
assert !(dir instanceof TrackingDirectoryWrapper);
|
||||
this.dir = dir;
|
||||
this.version = version;
|
||||
this.name = name;
|
||||
this.docCount = docCount;
|
||||
this.normGen = normGen;
|
||||
this.isCompoundFile = isCompoundFile;
|
||||
this.codec = codec;
|
||||
this.diagnostics = diagnostics;
|
||||
|
@ -120,17 +111,7 @@ public final class SegmentInfo {
|
|||
*/
|
||||
@Deprecated
|
||||
boolean hasSeparateNorms() {
|
||||
if (normGen == null) {
|
||||
return false;
|
||||
} else {
|
||||
for (long fieldNormGen : normGen.values()) {
|
||||
if (fieldNormGen >= YES) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
return getAttribute(Lucene3xSegmentInfoFormat.NORMGEN_KEY) != null;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -267,11 +248,6 @@ public final class SegmentInfo {
|
|||
return version;
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public Map<Integer,Long> getNormGen() {
|
||||
return normGen;
|
||||
}
|
||||
|
||||
private Set<String> setFiles;
|
||||
|
||||
public void setFiles(Set<String> files) {
|
||||
|
|
|
@ -429,16 +429,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
|
|||
|
||||
output.writeStringStringMap(si.attributes());
|
||||
|
||||
Map<Integer,Long> normGen = si.getNormGen();
|
||||
if (normGen == null) {
|
||||
output.writeInt(SegmentInfo.NO);
|
||||
} else {
|
||||
output.writeInt(normGen.size());
|
||||
for (Map.Entry<Integer,Long> entry : normGen.entrySet()) {
|
||||
output.writeLong(entry.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
|
||||
output.writeStringStringMap(si.getDiagnostics());
|
||||
output.writeStringSet(si.files());
|
||||
|
|
|
@ -256,8 +256,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
final Directory dir = newDirectory();
|
||||
this.write(fieldInfos, dir, fields, true);
|
||||
Codec codec = Codec.getDefault();
|
||||
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, null, false,
|
||||
codec, null, null);
|
||||
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
|
||||
|
||||
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random()), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
|
||||
|
||||
|
@ -314,7 +313,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
this.write(fieldInfos, dir, fields, false);
|
||||
Codec codec = Codec.getDefault();
|
||||
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000,
|
||||
null, false, codec, null, null);
|
||||
false, codec, null, null);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now read postings");
|
||||
|
@ -618,7 +617,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
|
||||
final int termIndexInterval = _TestUtil.nextInt(random(), 13, 27);
|
||||
final Codec codec = Codec.getDefault();
|
||||
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, null, false, codec, null, null);
|
||||
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
|
||||
final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, termIndexInterval, null, newIOContext(random()));
|
||||
|
||||
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state);
|
||||
|
|
|
@ -197,7 +197,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
|
||||
final Codec codec = Codec.getDefault();
|
||||
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
|
||||
final SegmentInfo si = new SegmentInfo(si1.info.dir, Constants.LUCENE_MAIN_VERSION, merged, -1, null, false, codec, null, null);
|
||||
final SegmentInfo si = new SegmentInfo(si1.info.dir, Constants.LUCENE_MAIN_VERSION, merged, -1, false, codec, null, null);
|
||||
|
||||
SegmentMerger merger = new SegmentMerger(si, InfoStream.getDefault(), trackingDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
|
||||
MergeState.CheckAbort.NONE, null, new FieldInfos.FieldNumbers(), context);
|
||||
|
@ -209,7 +209,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
r2.close();
|
||||
final SegmentInfo info = new SegmentInfo(si1.info.dir, Constants.LUCENE_MAIN_VERSION, merged,
|
||||
si1.info.getDocCount() + si2.info.getDocCount(),
|
||||
null, false, codec, null, null);
|
||||
false, codec, null, null);
|
||||
info.setFiles(new HashSet<String>(trackingDir.getCreatedFiles()));
|
||||
|
||||
if (useCompoundFile) {
|
||||
|
|
|
@ -77,7 +77,7 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
|
||||
public void testMerge() throws IOException {
|
||||
final Codec codec = Codec.getDefault();
|
||||
final SegmentInfo si = new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, -1, null, false, codec, null, null);
|
||||
final SegmentInfo si = new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, -1, false, codec, null, null);
|
||||
|
||||
SegmentMerger merger = new SegmentMerger(si, InfoStream.getDefault(), mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
|
||||
MergeState.CheckAbort.NONE, null, new FieldInfos.FieldNumbers(), newIOContext(random()));
|
||||
|
@ -89,7 +89,7 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
//Should be able to open a new SegmentReader against the new directory
|
||||
SegmentReader mergedReader = new SegmentReader(new SegmentInfoPerCommit(
|
||||
new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged,
|
||||
null, false, codec, null, null),
|
||||
false, codec, null, null),
|
||||
0, -1L),
|
||||
DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
|
||||
assertTrue(mergedReader != null);
|
||||
|
|
|
@ -146,7 +146,7 @@ public class IndexSplitter {
|
|||
SegmentInfo info = infoPerCommit.info;
|
||||
// Same info just changing the dir:
|
||||
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(),
|
||||
info.getNormGen(), info.getUseCompoundFile(),
|
||||
info.getUseCompoundFile(),
|
||||
info.getCodec(), info.getDiagnostics(), info.attributes());
|
||||
destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen()));
|
||||
// now copy files over
|
||||
|
|
Loading…
Reference in New Issue