LUCENE-5204: Make Directory easier to wrap.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1522661 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2013-09-12 16:59:34 +00:00
parent c3ccfbe8d1
commit d06ba3caaf
16 changed files with 270 additions and 335 deletions

View File

@ -53,6 +53,13 @@ New Features
for advanced use cases where String is too restrictive (Luca for advanced use cases where String is too restrictive (Luca
Cavanna, Robert Muir, Mike McCandless) Cavanna, Robert Muir, Mike McCandless)
Changes in backwards compatibility policy
* LUCENE-5204: Directory doesn't have default implementations for
LockFactory-related methods, which have been moved to BaseDirectory. If you
had a custom Directory implementation that extended Directory, you need to
extend BaseDirectory instead. (Adrien Grand)
======================= Lucene 4.5.0 ======================= ======================= Lucene 4.5.0 =======================
New features New features
@ -310,7 +317,7 @@ Changes in backwards compatibility policy
* LUCENE-5101: CachingWrapperFilter doesn't always return FixedBitSet instances * LUCENE-5101: CachingWrapperFilter doesn't always return FixedBitSet instances
anymore. Users of the join module can use anymore. Users of the join module can use
oal.search.join.FixedBitSetCachingWrapperFilter instead. (Adrien Grand) oal.search.join.FixedBitSetCachingWrapperFilter instead. (Adrien Grand)
Build Build
* SOLR-5159: Manifest includes non-parsed maven variables. * SOLR-5159: Manifest includes non-parsed maven variables.

View File

@ -0,0 +1,69 @@
package org.apache.lucene.store;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
/**
* Base implementation for a concrete {@link Directory}.
* @lucene.experimental
*/
public abstract class BaseDirectory extends Directory {
volatile protected boolean isOpen = true;
/** Holds the LockFactory instance (implements locking for
* this Directory instance). */
protected LockFactory lockFactory;
/** Sole constructor. */
protected BaseDirectory() {
super();
}
@Override
public Lock makeLock(String name) {
return lockFactory.makeLock(name);
}
@Override
public void clearLock(String name) throws IOException {
if (lockFactory != null) {
lockFactory.clearLock(name);
}
}
@Override
public void setLockFactory(LockFactory lockFactory) throws IOException {
assert lockFactory != null;
this.lockFactory = lockFactory;
lockFactory.setLockPrefix(this.getLockID());
}
@Override
public LockFactory getLockFactory() {
return this.lockFactory;
}
@Override
protected final void ensureOpen() throws AlreadyClosedException {
if (!isOpen)
throw new AlreadyClosedException("this Directory is closed");
}
}

View File

@ -71,7 +71,7 @@ import java.io.IOException;
* *
* @lucene.experimental * @lucene.experimental
*/ */
public final class CompoundFileDirectory extends Directory { public final class CompoundFileDirectory extends BaseDirectory {
/** Offset/Length for a slice inside of a compound file */ /** Offset/Length for a slice inside of a compound file */
public static final class FileEntry { public static final class FileEntry {

View File

@ -44,12 +44,6 @@ import org.apache.lucene.util.IOUtils;
*/ */
public abstract class Directory implements Closeable { public abstract class Directory implements Closeable {
volatile protected boolean isOpen = true;
/** Holds the LockFactory instance (implements locking for
* this Directory instance). */
protected LockFactory lockFactory;
/** /**
* Returns an array of strings, one for each file in the directory. * Returns an array of strings, one for each file in the directory.
* *
@ -115,20 +109,15 @@ public abstract class Directory implements Closeable {
/** Construct a {@link Lock}. /** Construct a {@link Lock}.
* @param name the name of the lock file * @param name the name of the lock file
*/ */
public Lock makeLock(String name) { public abstract Lock makeLock(String name);
return lockFactory.makeLock(name);
}
/** /**
* Attempt to clear (forcefully unlock and remove) the * Attempt to clear (forcefully unlock and remove) the
* specified lock. Only call this at a time when you are * specified lock. Only call this at a time when you are
* certain this lock is no longer in use. * certain this lock is no longer in use.
* @param name name of the lock to be cleared. * @param name name of the lock to be cleared.
*/ */
public void clearLock(String name) throws IOException { public abstract void clearLock(String name) throws IOException;
if (lockFactory != null) {
lockFactory.clearLock(name);
}
}
/** Closes the store. */ /** Closes the store. */
@Override @Override
@ -144,11 +133,7 @@ public abstract class Directory implements Closeable {
* *
* @param lockFactory instance of {@link LockFactory}. * @param lockFactory instance of {@link LockFactory}.
*/ */
public void setLockFactory(LockFactory lockFactory) throws IOException { public abstract void setLockFactory(LockFactory lockFactory) throws IOException;
assert lockFactory != null;
this.lockFactory = lockFactory;
lockFactory.setLockPrefix(this.getLockID());
}
/** /**
* Get the LockFactory that this Directory instance is * Get the LockFactory that this Directory instance is
@ -156,9 +141,7 @@ public abstract class Directory implements Closeable {
* may be null for Directory implementations that provide * may be null for Directory implementations that provide
* their own locking implementation. * their own locking implementation.
*/ */
public LockFactory getLockFactory() { public abstract LockFactory getLockFactory();
return this.lockFactory;
}
/** /**
* Return a string identifier that uniquely differentiates * Return a string identifier that uniquely differentiates
@ -252,10 +235,7 @@ public abstract class Directory implements Closeable {
/** /**
* @throws AlreadyClosedException if this Directory is closed * @throws AlreadyClosedException if this Directory is closed
*/ */
protected final void ensureOpen() throws AlreadyClosedException { protected void ensureOpen() throws AlreadyClosedException {}
if (!isOpen)
throw new AlreadyClosedException("this Directory is closed");
}
/** /**
* Allows to create one or more sliced {@link IndexInput} instances from a single * Allows to create one or more sliced {@link IndexInput} instances from a single

View File

@ -110,7 +110,7 @@ import org.apache.lucene.util.IOUtils;
* *
* @see Directory * @see Directory
*/ */
public abstract class FSDirectory extends Directory { public abstract class FSDirectory extends BaseDirectory {
protected final File directory; // The underlying filesystem directory protected final File directory; // The underlying filesystem directory
protected final Set<String> staleFiles = synchronizedSet(new HashSet<String>()); // Files written, but not yet sync'ed protected final Set<String> staleFiles = synchronizedSet(new HashSet<String>()); // Files written, but not yet sync'ed

View File

@ -39,7 +39,7 @@ import java.util.HashSet;
* @lucene.experimental * @lucene.experimental
*/ */
public class FileSwitchDirectory extends Directory { public class FileSwitchDirectory extends BaseDirectory {
private final Directory secondaryDir; private final Directory secondaryDir;
private final Directory primaryDir; private final Directory primaryDir;
private final Set<String> primaryExtensions; private final Set<String> primaryExtensions;

View File

@ -0,0 +1,113 @@
package org.apache.lucene.store;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Collection;
/** Directory implementation that delegates calls to another directory.
* This class can be used to add limitations on top of an existing
* {@link Directory} implementation such as
* {@link RateLimitedDirectoryWrapper rate limiting} or to add additional
* sanity checks for tests. However, if you plan to write your own
* {@link Directory} implementation, you should consider extending directly
* {@link Directory} or {@link BaseDirectory} rather than try to reuse
* functionality of existing {@link Directory}s by extending this class.
* @lucene.internal */
public class FilterDirectory extends Directory {
protected final Directory in;
/** Sole constructor, typically called from sub-classes. */
protected FilterDirectory(Directory in) {
this.in = in;
}
/** Return the wrapped {@link Directory}. */
public final Directory getDelegate() {
return in;
}
@Override
public String[] listAll() throws IOException {
return in.listAll();
}
@Override
public boolean fileExists(String name) throws IOException {
return in.fileExists(name);
}
@Override
public void deleteFile(String name) throws IOException {
in.deleteFile(name);
}
@Override
public long fileLength(String name) throws IOException {
return in.fileLength(name);
}
@Override
public IndexOutput createOutput(String name, IOContext context)
throws IOException {
return in.createOutput(name, context);
}
@Override
public void sync(Collection<String> names) throws IOException {
in.sync(names);
}
@Override
public IndexInput openInput(String name, IOContext context)
throws IOException {
return in.openInput(name, context);
}
@Override
public Lock makeLock(String name) {
return in.makeLock(name);
}
@Override
public void clearLock(String name) throws IOException {
in.clearLock(name);
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public void setLockFactory(LockFactory lockFactory) throws IOException {
in.setLockFactory(lockFactory);
}
@Override
public LockFactory getLockFactory() {
return in.getLockFactory();
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + in.toString() + ")";
}
}

View File

@ -45,7 +45,7 @@ import java.util.concurrent.atomic.AtomicLong;
* implementation working directly on the file system cache of the * implementation working directly on the file system cache of the
* operating system, so copying data to Java heap space is not useful. * operating system, so copying data to Java heap space is not useful.
*/ */
public class RAMDirectory extends Directory { public class RAMDirectory extends BaseDirectory {
protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>(); protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>();
protected final AtomicLong sizeInBytes = new AtomicLong(); protected final AtomicLong sizeInBytes = new AtomicLong();

View File

@ -16,7 +16,6 @@ package org.apache.lucene.store;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.store.IOContext.Context; import org.apache.lucene.store.IOContext.Context;
@ -28,123 +27,40 @@ import org.apache.lucene.store.IOContext.Context;
* @see #setRateLimiter(RateLimiter, IOContext.Context) * @see #setRateLimiter(RateLimiter, IOContext.Context)
* @lucene.experimental * @lucene.experimental
*/ */
public final class RateLimitedDirectoryWrapper extends Directory { public final class RateLimitedDirectoryWrapper extends FilterDirectory {
private final Directory delegate;
// we need to be volatile here to make sure we see all the values that are set // we need to be volatile here to make sure we see all the values that are set
// / modified concurrently // / modified concurrently
private volatile RateLimiter[] contextRateLimiters = new RateLimiter[IOContext.Context private volatile RateLimiter[] contextRateLimiters = new RateLimiter[IOContext.Context
.values().length]; .values().length];
public RateLimitedDirectoryWrapper(Directory wrapped) { public RateLimitedDirectoryWrapper(Directory wrapped) {
this.delegate = wrapped; super(wrapped);
}
public Directory getDelegate() {
return delegate;
}
@Override
public String[] listAll() throws IOException {
ensureOpen();
return delegate.listAll();
}
@Override
public boolean fileExists(String name) throws IOException {
ensureOpen();
return delegate.fileExists(name);
}
@Override
public void deleteFile(String name) throws IOException {
ensureOpen();
delegate.deleteFile(name);
}
@Override
public long fileLength(String name) throws IOException {
ensureOpen();
return delegate.fileLength(name);
} }
@Override @Override
public IndexOutput createOutput(String name, IOContext context) public IndexOutput createOutput(String name, IOContext context)
throws IOException { throws IOException {
ensureOpen(); ensureOpen();
final IndexOutput output = delegate.createOutput(name, context); final IndexOutput output = super.createOutput(name, context);
final RateLimiter limiter = getRateLimiter(context.context); final RateLimiter limiter = getRateLimiter(context.context);
if (limiter != null) { if (limiter != null) {
return new RateLimitedIndexOutput(limiter, output); return new RateLimitedIndexOutput(limiter, output);
} }
return output; return output;
} }
@Override
public void sync(Collection<String> names) throws IOException {
ensureOpen();
delegate.sync(names);
}
@Override
public IndexInput openInput(String name, IOContext context)
throws IOException {
ensureOpen();
return delegate.openInput(name, context);
}
@Override
public void close() throws IOException {
isOpen = false;
delegate.close();
}
@Override @Override
public IndexInputSlicer createSlicer(String name, IOContext context) public IndexInputSlicer createSlicer(String name, IOContext context)
throws IOException { throws IOException {
ensureOpen(); ensureOpen();
return delegate.createSlicer(name, context); return in.createSlicer(name, context);
}
@Override
public Lock makeLock(String name) {
ensureOpen();
return delegate.makeLock(name);
}
@Override
public void clearLock(String name) throws IOException {
ensureOpen();
delegate.clearLock(name);
}
@Override
public void setLockFactory(LockFactory lockFactory) throws IOException {
ensureOpen();
delegate.setLockFactory(lockFactory);
}
@Override
public LockFactory getLockFactory() {
ensureOpen();
return delegate.getLockFactory();
}
@Override
public String getLockID() {
ensureOpen();
return delegate.getLockID();
}
@Override
public String toString() {
return "RateLimitedDirectoryWrapper(" + delegate.toString() + ")";
} }
@Override @Override
public void copy(Directory to, String src, String dest, IOContext context) throws IOException { public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
ensureOpen(); ensureOpen();
delegate.copy(to, src, dest, context); in.copy(to, src, dest, context);
} }
private RateLimiter getRateLimiter(IOContext.Context context) { private RateLimiter getRateLimiter(IOContext.Context context) {

View File

@ -17,105 +17,42 @@ package org.apache.lucene.store;
* limitations under the License. * limitations under the License.
*/ */
import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Collection;
import java.util.HashSet; import java.util.HashSet;
import java.util.Set; import java.util.Set;
/** A delegating Directory that records which files were /** A delegating Directory that records which files were
* written to and deleted. */ * written to and deleted. */
public final class TrackingDirectoryWrapper extends Directory implements Closeable { public final class TrackingDirectoryWrapper extends FilterDirectory {
private final Directory other;
private final Set<String> createdFileNames = Collections.synchronizedSet(new HashSet<String>()); private final Set<String> createdFileNames = Collections.synchronizedSet(new HashSet<String>());
public TrackingDirectoryWrapper(Directory other) { public TrackingDirectoryWrapper(Directory in) {
this.other = other; super(in);
}
@Override
public String[] listAll() throws IOException {
return other.listAll();
}
@Override
public boolean fileExists(String name) throws IOException {
return other.fileExists(name);
} }
@Override @Override
public void deleteFile(String name) throws IOException { public void deleteFile(String name) throws IOException {
createdFileNames.remove(name); createdFileNames.remove(name);
other.deleteFile(name); in.deleteFile(name);
}
@Override
public long fileLength(String name) throws IOException {
return other.fileLength(name);
} }
@Override @Override
public IndexOutput createOutput(String name, IOContext context) throws IOException { public IndexOutput createOutput(String name, IOContext context) throws IOException {
createdFileNames.add(name); createdFileNames.add(name);
return other.createOutput(name, context); return in.createOutput(name, context);
}
@Override
public void sync(Collection<String> names) throws IOException {
other.sync(names);
}
@Override
public IndexInput openInput(String name, IOContext context) throws IOException {
return other.openInput(name, context);
}
@Override
public Lock makeLock(String name) {
return other.makeLock(name);
}
@Override
public void clearLock(String name) throws IOException {
other.clearLock(name);
}
@Override
public void close() throws IOException {
other.close();
}
@Override
public void setLockFactory(LockFactory lockFactory) throws IOException {
other.setLockFactory(lockFactory);
}
@Override
public LockFactory getLockFactory() {
return other.getLockFactory();
}
@Override
public String getLockID() {
return other.getLockID();
}
@Override
public String toString() {
return "TrackingDirectoryWrapper(" + other.toString() + ")";
} }
@Override @Override
public void copy(Directory to, String src, String dest, IOContext context) throws IOException { public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
createdFileNames.add(dest); createdFileNames.add(dest);
other.copy(to, src, dest, context); in.copy(to, src, dest, context);
} }
@Override @Override
public Directory.IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException { public Directory.IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
return other.createSlicer(name, context); return in.createSlicer(name, context);
} }
// maybe clone before returning.... all callers are // maybe clone before returning.... all callers are
@ -124,7 +61,4 @@ public final class TrackingDirectoryWrapper extends Directory implements Closeab
return createdFileNames; return createdFileNames;
} }
public Directory getDelegate() {
return other;
}
} }

View File

@ -27,6 +27,7 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.BaseDirectory;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
@ -146,7 +147,7 @@ public class TestCrashCausesCorruptIndex extends LuceneTestCase {
* This test class provides direct access to "simulating" a crash right after * This test class provides direct access to "simulating" a crash right after
* realDirectory.createOutput(..) has been called on a certain specified name. * realDirectory.createOutput(..) has been called on a certain specified name.
*/ */
private static class CrashAfterCreateOutput extends Directory { private static class CrashAfterCreateOutput extends BaseDirectory {
private Directory realDirectory; private Directory realDirectory;
private String crashAfterCreateOutput; private String crashAfterCreateOutput;

View File

@ -36,6 +36,7 @@ import org.apache.lucene.document.StringField;
import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.FieldCache; import org.apache.lucene.search.FieldCache;
import org.apache.lucene.store.BaseDirectory;
import org.apache.lucene.store.BufferedIndexInput; import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
@ -112,7 +113,7 @@ public class TestFieldsReader extends LuceneTestCase {
} }
public static class FaultyFSDirectory extends Directory { public static class FaultyFSDirectory extends BaseDirectory {
Directory fsDir; Directory fsDir;

View File

@ -283,7 +283,7 @@ public class TestBufferedIndexInput extends LuceneTestCase {
} }
} }
private static class MockFSDirectory extends Directory { private static class MockFSDirectory extends BaseDirectory {
List<IndexInput> allIndexInputs = new ArrayList<IndexInput>(); List<IndexInput> allIndexInputs = new ArrayList<IndexInput>();

View File

@ -18,7 +18,6 @@ package org.apache.lucene.store;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.util._TestUtil; import org.apache.lucene.util._TestUtil;
@ -29,15 +28,14 @@ import org.apache.lucene.util._TestUtil;
// do NOT make any methods in this class synchronized, volatile // do NOT make any methods in this class synchronized, volatile
// do NOT import anything from the concurrency package. // do NOT import anything from the concurrency package.
// no randoms, no nothing. // no randoms, no nothing.
public class BaseDirectoryWrapper extends Directory { public class BaseDirectoryWrapper extends FilterDirectory {
/** our in directory */
protected final Directory delegate;
private boolean checkIndexOnClose = true; private boolean checkIndexOnClose = true;
private boolean crossCheckTermVectorsOnClose = true; private boolean crossCheckTermVectorsOnClose = true;
protected volatile boolean isOpen = true;
public BaseDirectoryWrapper(Directory delegate) { public BaseDirectoryWrapper(Directory delegate) {
this.delegate = delegate; super(delegate);
} }
@Override @Override
@ -46,7 +44,7 @@ public class BaseDirectoryWrapper extends Directory {
if (checkIndexOnClose && DirectoryReader.indexExists(this)) { if (checkIndexOnClose && DirectoryReader.indexExists(this)) {
_TestUtil.checkIndex(this, crossCheckTermVectorsOnClose); _TestUtil.checkIndex(this, crossCheckTermVectorsOnClose);
} }
delegate.close(); super.close();
} }
public boolean isOpen() { public boolean isOpen() {
@ -73,80 +71,13 @@ public class BaseDirectoryWrapper extends Directory {
return crossCheckTermVectorsOnClose; return crossCheckTermVectorsOnClose;
} }
// directory methods: delegate
@Override
public String[] listAll() throws IOException {
return delegate.listAll();
}
@Override
public boolean fileExists(String name) throws IOException {
return delegate.fileExists(name);
}
@Override
public void deleteFile(String name) throws IOException {
delegate.deleteFile(name);
}
@Override
public long fileLength(String name) throws IOException {
return delegate.fileLength(name);
}
@Override
public IndexOutput createOutput(String name, IOContext context) throws IOException {
return delegate.createOutput(name, context);
}
@Override
public void sync(Collection<String> names) throws IOException {
delegate.sync(names);
}
@Override
public IndexInput openInput(String name, IOContext context) throws IOException {
return delegate.openInput(name, context);
}
@Override
public Lock makeLock(String name) {
return delegate.makeLock(name);
}
@Override
public void clearLock(String name) throws IOException {
delegate.clearLock(name);
}
@Override
public void setLockFactory(LockFactory lockFactory) throws IOException {
delegate.setLockFactory(lockFactory);
}
@Override
public LockFactory getLockFactory() {
return delegate.getLockFactory();
}
@Override
public String getLockID() {
return delegate.getLockID();
}
@Override
public String toString() {
return "BaseDirectoryWrapper(" + delegate.toString() + ")";
}
@Override @Override
public void copy(Directory to, String src, String dest, IOContext context) throws IOException { public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
delegate.copy(to, src, dest, context); in.copy(to, src, dest, context);
} }
@Override @Override
public IndexInputSlicer createSlicer(String name, IOContext context) throws IOException { public IndexInputSlicer createSlicer(String name, IOContext context) throws IOException {
return delegate.createSlicer(name, context); return in.createSlicer(name, context);
} }
} }

View File

@ -80,6 +80,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
volatile boolean crashed; volatile boolean crashed;
private ThrottledIndexOutput throttledOutput; private ThrottledIndexOutput throttledOutput;
private Throttling throttling = Throttling.SOMETIMES; private Throttling throttling = Throttling.SOMETIMES;
protected LockFactory lockFactory;
final AtomicInteger inputCloneCount = new AtomicInteger(); final AtomicInteger inputCloneCount = new AtomicInteger();
@ -121,10 +122,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
this.lockFactory = new MockLockFactoryWrapper(this, delegate.getLockFactory()); this.lockFactory = new MockLockFactoryWrapper(this, delegate.getLockFactory());
init(); init();
} }
public Directory getDelegate() {
return this.delegate;
}
public int getInputCloneCount() { public int getInputCloneCount() {
return inputCloneCount.get(); return inputCloneCount.get();
@ -160,21 +157,15 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
} }
/** /**
* Returns true if {@link #getDelegate() delegate} must sync its files. * Returns true if {@link #in} must sync its files.
* Currently, only {@link NRTCachingDirectory} requires sync'ing its files * Currently, only {@link NRTCachingDirectory} requires sync'ing its files
* because otherwise they are cached in an internal {@link RAMDirectory}. If * because otherwise they are cached in an internal {@link RAMDirectory}. If
* other directories require that too, they should be added to this method. * other directories require that too, they should be added to this method.
*/ */
private boolean mustSync() { private boolean mustSync() {
Directory delegate = this.delegate; Directory delegate = in;
while (true) { while (delegate instanceof FilterDirectory) {
if (delegate instanceof RateLimitedDirectoryWrapper) { delegate = ((FilterDirectory) delegate).getDelegate();
delegate = ((RateLimitedDirectoryWrapper) delegate).getDelegate();
} else if (delegate instanceof TrackingDirectoryWrapper) {
delegate = ((TrackingDirectoryWrapper) delegate).getDelegate();
} else {
break;
}
} }
return delegate instanceof NRTCachingDirectory; return delegate instanceof NRTCachingDirectory;
} }
@ -191,31 +182,22 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
for (String name : names) { for (String name : names) {
// randomly fail with IOE on any file // randomly fail with IOE on any file
maybeThrowIOException(name); maybeThrowIOException(name);
delegate.sync(Collections.singleton(name)); in.sync(Collections.singleton(name));
unSyncedFiles.remove(name); unSyncedFiles.remove(name);
} }
} else { } else {
unSyncedFiles.removeAll(names); unSyncedFiles.removeAll(names);
} }
} }
@Override
public String toString() {
// NOTE: do not maybeYield here, since it consumes
// randomness and can thus (unexpectedly during
// debugging) change the behavior of a seed
// maybeYield();
return "MockDirWrapper(" + delegate + ")";
}
public synchronized final long sizeInBytes() throws IOException { public synchronized final long sizeInBytes() throws IOException {
if (delegate instanceof RAMDirectory) if (in instanceof RAMDirectory)
return ((RAMDirectory) delegate).sizeInBytes(); return ((RAMDirectory) in).sizeInBytes();
else { else {
// hack // hack
long size = 0; long size = 0;
for (String file : delegate.listAll()) for (String file : in.listAll())
size += delegate.fileLength(file); size += in.fileLength(file);
return size; return size;
} }
} }
@ -252,7 +234,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
long length = fileLength(name); long length = fileLength(name);
byte[] zeroes = new byte[256]; byte[] zeroes = new byte[256];
long upto = 0; long upto = 0;
IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState)); IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
while(upto < length) { while(upto < length) {
final int limit = (int) Math.min(length-upto, zeroes.length); final int limit = (int) Math.min(length-upto, zeroes.length);
out.writeBytes(zeroes, 0, limit); out.writeBytes(zeroes, 0, limit);
@ -268,12 +250,12 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
String tempFileName; String tempFileName;
while (true) { while (true) {
tempFileName = ""+randomState.nextInt(); tempFileName = ""+randomState.nextInt();
if (!delegate.fileExists(tempFileName)) { if (!in.fileExists(tempFileName)) {
break; break;
} }
} }
final IndexOutput tempOut = delegate.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState)); final IndexOutput tempOut = in.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState));
IndexInput ii = delegate.openInput(name, LuceneTestCase.newIOContext(randomState)); IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState));
tempOut.copyBytes(ii, ii.length()/2); tempOut.copyBytes(ii, ii.length()/2);
tempOut.close(); tempOut.close();
ii.close(); ii.close();
@ -281,8 +263,8 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
// Delete original and copy bytes back: // Delete original and copy bytes back:
deleteFile(name, true); deleteFile(name, true);
final IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState)); final IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
ii = delegate.openInput(tempFileName, LuceneTestCase.newIOContext(randomState)); ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
out.copyBytes(ii, ii.length()); out.copyBytes(ii, ii.length());
out.close(); out.close();
ii.close(); ii.close();
@ -294,7 +276,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
action = "fully truncated"; action = "fully truncated";
// Totally truncate the file to zero bytes // Totally truncate the file to zero bytes
deleteFile(name, true); deleteFile(name, true);
IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState)); IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
out.setLength(0); out.setLength(0);
out.close(); out.close();
} }
@ -436,7 +418,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
openFilesDeleted.remove(name); openFilesDeleted.remove(name);
} }
} }
delegate.deleteFile(name); in.deleteFile(name);
} }
public synchronized Set<String> getOpenDeletedFiles() { public synchronized Set<String> getOpenDeletedFiles() {
@ -476,8 +458,8 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
unSyncedFiles.add(name); unSyncedFiles.add(name);
createdFiles.add(name); createdFiles.add(name);
if (delegate instanceof RAMDirectory) { if (in instanceof RAMDirectory) {
RAMDirectory ramdir = (RAMDirectory) delegate; RAMDirectory ramdir = (RAMDirectory) in;
RAMFile file = new RAMFile(ramdir); RAMFile file = new RAMFile(ramdir);
RAMFile existing = ramdir.fileMap.get(name); RAMFile existing = ramdir.fileMap.get(name);
@ -493,7 +475,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
} }
} }
//System.out.println(Thread.currentThread().getName() + ": MDW: create " + name); //System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
IndexOutput delegateOutput = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState, context)); IndexOutput delegateOutput = in.createOutput(name, LuceneTestCase.newIOContext(randomState, context));
if (randomState.nextInt(10) == 0){ if (randomState.nextInt(10) == 0){
// once in a while wrap the IO in a Buffered IO with random buffer sizes // once in a while wrap the IO in a Buffered IO with random buffer sizes
delegateOutput = new BufferedIndexOutputWrapper(1+randomState.nextInt(BufferedIndexOutput.DEFAULT_BUFFER_SIZE), delegateOutput); delegateOutput = new BufferedIndexOutputWrapper(1+randomState.nextInt(BufferedIndexOutput.DEFAULT_BUFFER_SIZE), delegateOutput);
@ -504,7 +486,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
// throttling REALLY slows down tests, so don't do it very often for SOMETIMES. // throttling REALLY slows down tests, so don't do it very often for SOMETIMES.
if (throttling == Throttling.ALWAYS || if (throttling == Throttling.ALWAYS ||
(throttling == Throttling.SOMETIMES && randomState.nextInt(50) == 0) && !(delegate instanceof RateLimitedDirectoryWrapper)) { (throttling == Throttling.SOMETIMES && randomState.nextInt(50) == 0) && !(in instanceof RateLimitedDirectoryWrapper)) {
if (LuceneTestCase.VERBOSE) { if (LuceneTestCase.VERBOSE) {
System.out.println("MockDirectoryWrapper: throttling indexOutput (" + name + ")"); System.out.println("MockDirectoryWrapper: throttling indexOutput (" + name + ")");
} }
@ -544,8 +526,8 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
if (failOnOpenInput) { if (failOnOpenInput) {
maybeThrowDeterministicException(); maybeThrowDeterministicException();
} }
if (!delegate.fileExists(name)) { if (!in.fileExists(name)) {
throw randomState.nextBoolean() ? new FileNotFoundException(name + " in dir=" + delegate) : new NoSuchFileException(name + " in dir=" + delegate); throw randomState.nextBoolean() ? new FileNotFoundException(name + " in dir=" + in) : new NoSuchFileException(name + " in dir=" + in);
} }
// cannot open a file for input if it's still open for // cannot open a file for input if it's still open for
@ -554,7 +536,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false); throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
} }
IndexInput delegateInput = delegate.openInput(name, LuceneTestCase.newIOContext(randomState, context)); IndexInput delegateInput = in.openInput(name, LuceneTestCase.newIOContext(randomState, context));
final IndexInput ii; final IndexInput ii;
int randomInt = randomState.nextInt(500); int randomInt = randomState.nextInt(500);
@ -577,10 +559,10 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
/** Provided for testing purposes. Use sizeInBytes() instead. */ /** Provided for testing purposes. Use sizeInBytes() instead. */
public synchronized final long getRecomputedSizeInBytes() throws IOException { public synchronized final long getRecomputedSizeInBytes() throws IOException {
if (!(delegate instanceof RAMDirectory)) if (!(in instanceof RAMDirectory))
return sizeInBytes(); return sizeInBytes();
long size = 0; long size = 0;
for(final RAMFile file: ((RAMDirectory)delegate).fileMap.values()) { for(final RAMFile file: ((RAMDirectory)in).fileMap.values()) {
size += file.getSizeInBytes(); size += file.getSizeInBytes();
} }
return size; return size;
@ -593,10 +575,10 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
*/ */
public final synchronized long getRecomputedActualSizeInBytes() throws IOException { public final synchronized long getRecomputedActualSizeInBytes() throws IOException {
if (!(delegate instanceof RAMDirectory)) if (!(in instanceof RAMDirectory))
return sizeInBytes(); return sizeInBytes();
long size = 0; long size = 0;
for (final RAMFile file : ((RAMDirectory)delegate).fileMap.values()) for (final RAMFile file : ((RAMDirectory)in).fileMap.values())
size += file.length; size += file.length;
return size; return size;
} }
@ -666,8 +648,8 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
String[] startFiles = allFiles.toArray(new String[0]); String[] startFiles = allFiles.toArray(new String[0]);
IndexWriterConfig iwc = new IndexWriterConfig(LuceneTestCase.TEST_VERSION_CURRENT, null); IndexWriterConfig iwc = new IndexWriterConfig(LuceneTestCase.TEST_VERSION_CURRENT, null);
iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE); iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
new IndexWriter(delegate, iwc).rollback(); new IndexWriter(in, iwc).rollback();
String[] endFiles = delegate.listAll(); String[] endFiles = in.listAll();
Set<String> startSet = new TreeSet<String>(Arrays.asList(startFiles)); Set<String> startSet = new TreeSet<String>(Arrays.asList(startFiles));
Set<String> endSet = new TreeSet<String>(Arrays.asList(endFiles)); Set<String> endSet = new TreeSet<String>(Arrays.asList(endFiles));
@ -692,13 +674,13 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
} }
SegmentInfos sis = new SegmentInfos(); SegmentInfos sis = new SegmentInfos();
try { try {
sis.read(delegate, file); sis.read(in, file);
} catch (IOException ioe) { } catch (IOException ioe) {
// OK: likely some of the .si files were deleted // OK: likely some of the .si files were deleted
} }
try { try {
Set<String> ghosts = new HashSet<String>(sis.files(delegate, false)); Set<String> ghosts = new HashSet<String>(sis.files(in, false));
for (String s : ghosts) { for (String s : ghosts) {
if (endSet.contains(s) && !startSet.contains(s)) { if (endSet.contains(s) && !startSet.contains(s)) {
assert pendingDeletions.contains(s); assert pendingDeletions.contains(s);
@ -763,7 +745,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
} }
} }
} }
delegate.close(); in.close();
} }
synchronized void removeOpenFile(Closeable c, String name) { synchronized void removeOpenFile(Closeable c, String name) {
@ -856,19 +838,19 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
@Override @Override
public synchronized String[] listAll() throws IOException { public synchronized String[] listAll() throws IOException {
maybeYield(); maybeYield();
return delegate.listAll(); return in.listAll();
} }
@Override @Override
public synchronized boolean fileExists(String name) throws IOException { public synchronized boolean fileExists(String name) throws IOException {
maybeYield(); maybeYield();
return delegate.fileExists(name); return in.fileExists(name);
} }
@Override @Override
public synchronized long fileLength(String name) throws IOException { public synchronized long fileLength(String name) throws IOException {
maybeYield(); maybeYield();
return delegate.fileLength(name); return in.fileLength(name);
} }
@Override @Override
@ -888,7 +870,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
maybeYield(); maybeYield();
// sneaky: we must pass the original this way to the dir, because // sneaky: we must pass the original this way to the dir, because
// some impls (e.g. FSDir) do instanceof here. // some impls (e.g. FSDir) do instanceof here.
delegate.setLockFactory(lockFactory); in.setLockFactory(lockFactory);
// now set our wrapped factory here // now set our wrapped factory here
this.lockFactory = new MockLockFactoryWrapper(this, lockFactory); this.lockFactory = new MockLockFactoryWrapper(this, lockFactory);
} }
@ -899,28 +881,28 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
if (wrapLockFactory) { if (wrapLockFactory) {
return lockFactory; return lockFactory;
} else { } else {
return delegate.getLockFactory(); return in.getLockFactory();
} }
} }
@Override @Override
public synchronized String getLockID() { public synchronized String getLockID() {
maybeYield(); maybeYield();
return delegate.getLockID(); return in.getLockID();
} }
@Override @Override
public synchronized void copy(Directory to, String src, String dest, IOContext context) throws IOException { public synchronized void copy(Directory to, String src, String dest, IOContext context) throws IOException {
maybeYield(); maybeYield();
// randomize the IOContext here? // randomize the IOContext here?
delegate.copy(to, src, dest, context); in.copy(to, src, dest, context);
} }
@Override @Override
public IndexInputSlicer createSlicer(final String name, IOContext context) public IndexInputSlicer createSlicer(final String name, IOContext context)
throws IOException { throws IOException {
maybeYield(); maybeYield();
if (!delegate.fileExists(name)) { if (!in.fileExists(name)) {
throw randomState.nextBoolean() ? new FileNotFoundException(name) : new NoSuchFileException(name); throw randomState.nextBoolean() ? new FileNotFoundException(name) : new NoSuchFileException(name);
} }
// cannot open a file for input if it's still open for // cannot open a file for input if it's still open for
@ -929,7 +911,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false); throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
} }
final IndexInputSlicer delegateHandle = delegate.createSlicer(name, context); final IndexInputSlicer delegateHandle = in.createSlicer(name, context);
final IndexInputSlicer handle = new IndexInputSlicer() { final IndexInputSlicer handle = new IndexInputSlicer() {
private boolean isClosed; private boolean isClosed;

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.lucene.store.BaseDirectory;
import org.apache.lucene.store.BufferedIndexOutput; import org.apache.lucene.store.BufferedIndexOutput;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
@ -39,7 +40,7 @@ import org.apache.solr.store.blockcache.CustomBufferedIndexInput;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
public class HdfsDirectory extends Directory { public class HdfsDirectory extends BaseDirectory {
public static Logger LOG = LoggerFactory.getLogger(HdfsDirectory.class); public static Logger LOG = LoggerFactory.getLogger(HdfsDirectory.class);
public static final int BUFFER_SIZE = 8192; public static final int BUFFER_SIZE = 8192;