mirror of https://github.com/apache/lucene.git
LUCENE-6266: Remove unnecessary Directory parameters from SIS/SIPC/SI
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1661172 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
bd89330adf
commit
8607f76503
|
@ -134,6 +134,9 @@ API Changes
|
|||
* LUCENE-6268: Replace FieldValueFilter and DocValuesRangeFilter with equivalent
|
||||
queries that support approximations. (Adrien Grand)
|
||||
|
||||
* LUCENE-6266: Remove unnecessary Directory params from SegmentInfo.toString,
|
||||
SegmentInfos.files/toString, and SegmentCommitInfo.toString. (Robert Muir)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-6248: Remove unused odd constants from StandardSyntaxParser.jj
|
||||
|
|
|
@ -334,7 +334,7 @@ class BufferedUpdatesStream implements Accountable {
|
|||
} else {
|
||||
dir = null;
|
||||
}
|
||||
infoStream.message("BD", "prune sis=" + segmentInfos.toString(dir) + " minGen=" + minGen + " packetCount=" + updates.size());
|
||||
infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + updates.size());
|
||||
}
|
||||
final int limit = updates.size();
|
||||
for(int delIDX=0;delIDX<limit;delIDX++) {
|
||||
|
|
|
@ -583,7 +583,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
}
|
||||
|
||||
// Save files so we can decr on next checkpoint/commit:
|
||||
lastFiles.addAll(segmentInfos.files(directory, false));
|
||||
lastFiles.addAll(segmentInfos.files(false));
|
||||
}
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
long t1 = System.nanoTime();
|
||||
|
@ -595,7 +595,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
assert locked();
|
||||
// If this is a commit point, also incRef the
|
||||
// segments_N file:
|
||||
for(final String fileName: segmentInfos.files(directory, isCommit)) {
|
||||
for(final String fileName: segmentInfos.files(isCommit)) {
|
||||
incRef(fileName);
|
||||
}
|
||||
}
|
||||
|
@ -670,7 +670,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
|
||||
void decRef(SegmentInfos segmentInfos) throws IOException {
|
||||
assert locked();
|
||||
decRef(segmentInfos.files(directory, false));
|
||||
decRef(segmentInfos.files(false));
|
||||
}
|
||||
|
||||
public boolean exists(String fileName) {
|
||||
|
@ -802,7 +802,7 @@ final class IndexFileDeleter implements Closeable {
|
|||
userData = segmentInfos.getUserData();
|
||||
segmentsFileName = segmentInfos.getSegmentsFileName();
|
||||
generation = segmentInfos.getGeneration();
|
||||
files = Collections.unmodifiableCollection(segmentInfos.files(directory, true));
|
||||
files = Collections.unmodifiableCollection(segmentInfos.files(true));
|
||||
segmentCount = segmentInfos.size();
|
||||
}
|
||||
|
||||
|
|
|
@ -1484,7 +1484,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
// for test purpose
|
||||
final synchronized Collection<String> getIndexFileNames() throws IOException {
|
||||
return segmentInfos.files(directory, true);
|
||||
return segmentInfos.files(true);
|
||||
}
|
||||
|
||||
// for test purpose
|
||||
|
@ -2689,7 +2689,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// we are trying to sync all referenced files, a
|
||||
// merge completes which would otherwise have
|
||||
// removed the files we are now syncing.
|
||||
filesToCommit = toCommit.files(directory, false);
|
||||
filesToCommit = toCommit.files(false);
|
||||
deleter.incRef(filesToCommit);
|
||||
}
|
||||
success = true;
|
||||
|
@ -4137,7 +4137,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
*
|
||||
* @lucene.internal */
|
||||
public synchronized String segString(SegmentCommitInfo info) {
|
||||
return info.toString(info.info.dir, numDeletedDocs(info) - info.getDelCount());
|
||||
return info.toString(numDeletedDocs(info) - info.getDelCount());
|
||||
}
|
||||
|
||||
private synchronized void doWait() {
|
||||
|
@ -4170,7 +4170,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
// called only from assert
|
||||
private boolean filesExist(SegmentInfos toSync) throws IOException {
|
||||
|
||||
Collection<String> files = toSync.files(directory, false);
|
||||
Collection<String> files = toSync.files(false);
|
||||
for(final String fileName: files) {
|
||||
assert slowFileExists(directory, fileName): "file " + fileName + " does not exist; files=" + Arrays.toString(directory.listAll());
|
||||
// If this trips it means we are missing a call to
|
||||
|
@ -4275,7 +4275,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
boolean success = false;
|
||||
final Collection<String> filesToSync;
|
||||
try {
|
||||
filesToSync = toSync.files(directory, false);
|
||||
filesToSync = toSync.files(false);
|
||||
directory.sync(filesToSync);
|
||||
success = true;
|
||||
} finally {
|
||||
|
|
|
@ -317,10 +317,19 @@ public class SegmentCommitInfo {
|
|||
}
|
||||
this.delCount = delCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a description of this segment.
|
||||
* @deprecated Use {@link #toString(int)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public String toString(Directory dir, int pendingDelCount) {
|
||||
return toString(pendingDelCount);
|
||||
}
|
||||
|
||||
/** Returns a description of this segment. */
|
||||
public String toString(Directory dir, int pendingDelCount) {
|
||||
String s = info.toString(dir, delCount + pendingDelCount);
|
||||
public String toString(int pendingDelCount) {
|
||||
String s = info.toString(delCount + pendingDelCount);
|
||||
if (delGen != -1) {
|
||||
s += ":delGen=" + delGen;
|
||||
}
|
||||
|
@ -335,7 +344,7 @@ public class SegmentCommitInfo {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return toString(info.dir, 0);
|
||||
return toString(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -169,7 +169,18 @@ public final class SegmentInfo {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return toString(dir, 0);
|
||||
return toString(0);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Used for debugging.
|
||||
*
|
||||
* @deprecated Use {@link #toString(int)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public String toString(Directory dir, int delCount) {
|
||||
return toString(delCount);
|
||||
}
|
||||
|
||||
/** Used for debugging. Format may suddenly change.
|
||||
|
@ -182,15 +193,12 @@ public final class SegmentInfo {
|
|||
* has 45 documents; it has 4 deletions (this part is
|
||||
* left off when there are no deletions).</p>
|
||||
*/
|
||||
public String toString(Directory dir, int delCount) {
|
||||
public String toString(int delCount) {
|
||||
StringBuilder s = new StringBuilder();
|
||||
s.append(name).append('(').append(version == null ? "?" : version).append(')').append(':');
|
||||
char cfs = getUseCompoundFile() ? 'c' : 'C';
|
||||
s.append(cfs);
|
||||
|
||||
if (this.dir != dir) {
|
||||
s.append('x');
|
||||
}
|
||||
s.append(docCount);
|
||||
|
||||
if (delCount != 0) {
|
||||
|
|
|
@ -621,13 +621,20 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
}
|
||||
write(dir);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all file names referenced by SegmentInfo.
|
||||
* @deprecated Use {@link #files(boolean)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public final Collection<String> files(Directory dir, boolean includeSegmentsFile) throws IOException {
|
||||
return files(includeSegmentsFile);
|
||||
}
|
||||
|
||||
/** Returns all file names referenced by SegmentInfo
|
||||
* instances matching the provided Directory (ie files
|
||||
* associated with any "external" segments are skipped).
|
||||
/** Returns all file names referenced by SegmentInfo.
|
||||
* The returned collection is recomputed on each
|
||||
* invocation. */
|
||||
public Collection<String> files(Directory dir, boolean includeSegmentsFile) throws IOException {
|
||||
public Collection<String> files(boolean includeSegmentsFile) throws IOException {
|
||||
HashSet<String> files = new HashSet<>();
|
||||
if (includeSegmentsFile) {
|
||||
final String segmentFileName = getSegmentsFileName();
|
||||
|
@ -638,10 +645,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
final int size = size();
|
||||
for(int i=0;i<size;i++) {
|
||||
final SegmentCommitInfo info = info(i);
|
||||
assert info.info.dir == dir;
|
||||
if (info.info.dir == dir) {
|
||||
files.addAll(info.files());
|
||||
}
|
||||
files.addAll(info.files());
|
||||
}
|
||||
|
||||
return files;
|
||||
|
@ -682,9 +686,19 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
prepareCommit(dir);
|
||||
finishCommit(dir);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns readable description of this segment.
|
||||
* @deprecated Use {@link #toString()} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public String toString(Directory dir) {
|
||||
return toString();
|
||||
}
|
||||
|
||||
/** Returns readable description of this segment. */
|
||||
public String toString(Directory directory) {
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder buffer = new StringBuilder();
|
||||
buffer.append(getSegmentsFileName()).append(": ");
|
||||
final int count = size();
|
||||
|
@ -693,7 +707,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
buffer.append(' ');
|
||||
}
|
||||
final SegmentCommitInfo info = info(i);
|
||||
buffer.append(info.toString(directory, 0));
|
||||
buffer.append(info.toString(0));
|
||||
}
|
||||
return buffer.toString();
|
||||
}
|
||||
|
|
|
@ -232,7 +232,7 @@ public final class SegmentReader extends CodecReader {
|
|||
public String toString() {
|
||||
// SegmentInfo.toString takes dir and number of
|
||||
// *pending* deletions; so we reverse compute that here:
|
||||
return si.toString(si.info.dir, si.info.getDocCount() - numDocs - si.getDelCount());
|
||||
return si.toString(si.info.getDocCount() - numDocs - si.getDelCount());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -393,7 +393,7 @@ final class StandardDirectoryReader extends DirectoryReader {
|
|||
segmentsFileName = infos.getSegmentsFileName();
|
||||
this.dir = dir;
|
||||
userData = infos.getUserData();
|
||||
files = Collections.unmodifiableCollection(infos.files(dir, true));
|
||||
files = Collections.unmodifiableCollection(infos.files(true));
|
||||
generation = infos.getGeneration();
|
||||
segmentCount = infos.size();
|
||||
}
|
||||
|
|
|
@ -241,7 +241,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
w.close();
|
||||
if (DEBUG) System.out.println("TEST: after close writer index=" + SegmentInfos.readLatestCommit(indexDir).toString(indexDir));
|
||||
if (DEBUG) System.out.println("TEST: after close writer index=" + SegmentInfos.readLatestCommit(indexDir));
|
||||
|
||||
/*
|
||||
DirectoryReader r = mgr.acquire();
|
||||
|
|
|
@ -824,7 +824,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
|||
}
|
||||
|
||||
try {
|
||||
Set<String> ghosts = new HashSet<>(sis.files(in, false));
|
||||
Set<String> ghosts = new HashSet<>(sis.files(false));
|
||||
for (String s : ghosts) {
|
||||
if (endSet.contains(s) && !startSet.contains(s)) {
|
||||
assert pendingDeletions.contains(s);
|
||||
|
|
|
@ -572,7 +572,7 @@ public class SnapPuller {
|
|||
private boolean hasUnusedFiles(Directory indexDir, IndexCommit commit) throws IOException {
|
||||
String segmentsFileName = commit.getSegmentsFileName();
|
||||
SegmentInfos infos = SegmentInfos.readCommit(indexDir, segmentsFileName);
|
||||
Set<String> currentFiles = new HashSet<>(infos.files(indexDir, true));
|
||||
Set<String> currentFiles = new HashSet<>(infos.files(true));
|
||||
String[] allFiles = indexDir.listAll();
|
||||
for (String file : allFiles) {
|
||||
if (!file.equals(segmentsFileName) && !currentFiles.contains(file) && !file.endsWith(".lock")) {
|
||||
|
|
Loading…
Reference in New Issue