-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
- Loading branch information
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,6 +21,7 @@ | |
|
||
import java.io.Closeable; | ||
import java.io.IOException; | ||
import java.lang.invoke.MethodHandles; | ||
import java.nio.file.NoSuchFileException; | ||
import java.nio.file.Path; | ||
import java.util.AbstractMap; | ||
|
@@ -31,6 +32,7 @@ | |
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.concurrent.BlockingQueue; | ||
import java.util.concurrent.CancellationException; | ||
import java.util.concurrent.ExecutionException; | ||
import java.util.concurrent.ExecutorService; | ||
|
@@ -49,15 +51,21 @@ | |
import org.apache.lucene.util.IOUtils; | ||
import org.apache.solr.common.util.CollectionUtil; | ||
import org.apache.solr.util.IOFunction; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public class TeeDirectory extends BaseDirectory { | ||
|
||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); | ||
|
||
private volatile Directory access; | ||
private final ExecutorService ioExec; | ||
private final AutoCloseable closeLocal; | ||
private final IOFunction<Void, Map.Entry<String, Directory>> accessFunction; | ||
private final IOFunction<Directory, Map.Entry<Directory, List<String>>> persistentFunction; | ||
private volatile Directory persistent; | ||
private final BlockingQueue<TeeDirectoryFactory.PersistentLengthVerification> | ||
persistentLengthVerificationQueue; | ||
|
||
/** | ||
* This ctor (with default inline config) exists to be invoked during testing, via | ||
|
@@ -66,8 +74,9 @@ public class TeeDirectory extends BaseDirectory { | |
public TeeDirectory(Path path, LockFactory lockFactory) throws IOException { | ||
super(TEE_LOCK_FACTORY); | ||
TeeDirectoryFactory.NodeLevelTeeDirectoryState ownState = | ||
new TeeDirectoryFactory.NodeLevelTeeDirectoryState(); | ||
new TeeDirectoryFactory.NodeLevelTeeDirectoryState(64); | ||
this.ioExec = ownState.ioExec; | ||
this.persistentLengthVerificationQueue = ownState.persistentLengthVerificationQueue; | ||
Directory naive = new MMapDirectory(path, lockFactory, MMapDirectory.DEFAULT_MAX_CHUNK_SIZE); | ||
this.access = naive; | ||
Path compressedPath = path; | ||
|
@@ -100,13 +109,14 @@ public TeeDirectory( | |
Directory naive, | ||
IOFunction<Void, Map.Entry<String, Directory>> accessFunction, | ||
IOFunction<Directory, Map.Entry<Directory, List<String>>> persistentFunction, | ||
ExecutorService ioExec) { | ||
TeeDirectoryFactory.NodeLevelTeeDirectoryState nodeLevelState) { | ||
super(TEE_LOCK_FACTORY); | ||
this.accessFunction = accessFunction; | ||
this.persistentFunction = persistentFunction; | ||
this.access = naive; | ||
this.ioExec = ioExec; | ||
this.ioExec = nodeLevelState.ioExec; | ||
this.closeLocal = null; | ||
this.persistentLengthVerificationQueue = nodeLevelState.persistentLengthVerificationQueue; | ||
} | ||
|
||
private List<String> associatedPaths; | ||
|
@@ -315,6 +325,19 @@ public void deleteFile(String name) throws IOException { | |
} finally { | ||
try { | ||
access.deleteFile(name); | ||
if (th instanceof NoSuchFileException) { | ||
// if we successfully delete the access copy, but threw `NoSuchFileException` for | ||
// the persistent copy, then swallow the original exception. | ||
// we expect this to happen only in the case of recovering after a disorderly shutdown | ||
// (or similar situation?). It is expected in such cases that a file may have been | ||
// flushed to disk for the access copy, and not for the persistent copy. In the case | ||
This comment has been minimized.
Sorry, something went wrong.
This comment has been minimized.
Sorry, something went wrong.
magibney
Author
Collaborator
|
||
// of `pending_segments_*` files, these files are ~explicitly partial, but must be | ||
// deleted. It's not ideal that we have to be lenient here, but we kind of have to do, | ||
// because we know there are legit situations where files can exist in access and not | ||
// in persistent, and we must support the ability to delete such files. | ||
log.info("swallow exception deleting missing persistent file: {}", name); | ||
th = null; | ||
} | ||
} catch (NoSuchFileException ex) { | ||
// when `persistent != null`, `access` is a special case. Since access may be on ephemeral | ||
// storage, we should be ok with files being already absent if we're asked to delete them. | ||
|
@@ -525,7 +548,11 @@ public void rename(String source, String dest) throws IOException { | |
|
||
@Override | ||
public IndexInput openInput(String name, IOContext context) throws IOException { | ||
return access.openInput(name, context); | ||
IndexInput ret = access.openInput(name, context); | ||
persistentLengthVerificationQueue.offer( | ||
new TeeDirectoryFactory.PersistentLengthVerification( | ||
access, persistent, name, ret.length())); | ||
return ret; | ||
} | ||
|
||
@Override | ||
|
File flush should happen in persistent directory first. right? do we do parallel flush?