From ba3292798f0c9a5c888f9f2b7fb3ab5b5b664599 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 19 Oct 2015 14:32:13 +0000 Subject: [PATCH] LUCENE-6843: RAMDirectory.listAll() should not return null file names. git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1709421 13f79535-47bb-0310-9956-ffa450edef68 --- .../src/java/org/apache/lucene/store/RAMDirectory.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java b/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java index fea0d954ec5..e3a60170e82 100644 --- a/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java +++ b/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import org.apache.lucene.util.Accountable; @@ -111,7 +110,13 @@ public class RAMDirectory extends BaseDirectory implements Accountable { // and do not synchronize or anything stronger. it's great for testing! // 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. - return fileMap.keySet().toArray(new String[fileMap.size()]); + // NOTE: don't replace this with return names.toArray(new String[names.size()]); + // or some files could be null at the end of the array if files are being deleted + // concurrently + Set fileNames = fileMap.keySet(); + List names = new ArrayList<>(fileNames.size()); + for (String name : fileNames) names.add(name); + return names.toArray(new String[names.size()]); } public final boolean fileNameExists(String name) {