LUCENE-2779: fix listAll()

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1041039 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Shai Erera 2010-12-01 15:03:27 +00:00
parent 86d8937f3a
commit 49508c277b
1 changed files with 12 additions and 3 deletions

View File

@ -20,8 +20,11 @@ package org.apache.lucene.store;
import java.io.IOException;
import java.io.FileNotFoundException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
@ -36,7 +39,7 @@ public class RAMDirectory extends Directory implements Serializable {
private static final long serialVersionUID = 1l;
protected Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>();
protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>();
protected final AtomicLong sizeInBytes = new AtomicLong();
// *****
@ -81,7 +84,12 @@ public class RAMDirectory extends Directory implements Serializable {
@Override
public final String[] listAll() {
ensureOpen();
return fileMap.keySet().toArray(new String[0]);
// NOTE: fileMap.keySet().toArray(new String[0]) is broken in non Sun JDKs,
// and the code below is resilient to map changes during the array population.
Set<String> fileNames = fileMap.keySet();
List<String> names = new ArrayList<String>(fileNames.size());
for (String name : fileNames) names.add(name);
return names.toArray(new String[names.size()]);
}
/** Returns true iff the named file exists in this directory. */
@ -188,6 +196,7 @@ public class RAMDirectory extends Directory implements Serializable {
return new RAMFile(this);
}
@Override
public void sync(Collection<String> names) throws IOException {
}
@ -206,6 +215,6 @@ public class RAMDirectory extends Directory implements Serializable {
@Override
public void close() {
isOpen = false;
fileMap = null;
fileMap.clear();
}
}