You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/12/01 13:12:38 UTC

svn commit: r1209047 - in /lucene/dev/trunk/lucene/src: java/org/apache/lucene/index/ java/org/apache/lucene/index/codecs/ java/org/apache/lucene/index/codecs/simpletext/ test/org/apache/lucene/index/

Author: rmuir
Date: Thu Dec  1 12:12:36 2011
New Revision: 1209047

URL: http://svn.apache.org/viewvc?rev=1209047&view=rev
Log:
LUCENE-3612: remove _X.fnx

Removed:
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java
Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java Thu Dec  1 12:12:36 2011
@@ -68,9 +68,6 @@ public final class IndexFileNames {
   /** Extension of separate norms */
   public static final String SEPARATE_NORMS_EXTENSION = "s";
 
-  /** Extension of global field numbers */
-  public static final String GLOBAL_FIELD_NUM_MAP_EXTENSION = "fnx";
-
   /**
    * This array contains all filename extensions used by
    * Lucene's index files, with one exception, namely the
@@ -85,7 +82,6 @@ public final class IndexFileNames {
     GEN_EXTENSION,
     NORMS_EXTENSION,
     COMPOUND_FILE_STORE_EXTENSION,
-    GLOBAL_FIELD_NUM_MAP_EXTENSION,
   };
 
   public static final String[] NON_STORE_INDEX_EXTENSIONS = new String[] {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Thu Dec  1 12:12:36 2011
@@ -882,7 +882,7 @@ public class IndexWriter implements Clos
       rollbackSegments = segmentInfos.createBackupSegmentInfos(true);
 
       // start with previous field numbers, but new FieldInfos
-      globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory);
+      globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap();
       docWriter = new DocumentsWriter(codec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
 
       // Default deleter (for backwards compatibility) is

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Thu Dec  1 12:12:36 2011
@@ -73,11 +73,6 @@ public final class SegmentInfos implemen
    */
   public long version = System.currentTimeMillis();
   
-  private long globalFieldMapVersion = 0; // version of the GFNM for the next commit
-  private long lastGlobalFieldMapVersion = 0; // version of the GFNM file we last successfully read or wrote
-  private long pendingMapVersion = -1; // version of the GFNM itself that we have last successfully written
-                                       // or -1 if we it was not written. This is set during prepareCommit 
-
   private long generation = 0;     // generation of the "segments_N" for the next commit
   private long lastGeneration = 0; // generation of the "segments_N" file we last successfully read
                                    // or wrote; this is normally the same as generation except if
@@ -184,15 +179,6 @@ public final class SegmentInfos implemen
                                                  lastGeneration);
   }
   
-  private String getGlobalFieldNumberName(long version) {
-    /*
-     * This creates a file name ${version}.fnx without a leading underscore
-     * since this file might belong to more than one segment (global map) and
-     * could otherwise easily be confused with a per-segment file.
-     */
-    return IndexFileNames.segmentFileName("_"+ version, "", IndexFileNames.GLOBAL_FIELD_NUM_MAP_EXTENSION);
-  }
-
   /**
    * Parse the generation off the segments file name and
    * return it.
@@ -297,8 +283,6 @@ public final class SegmentInfos implemen
         return null;
       }
     }.run();
-    // either we are on 4.0 or we don't have a lastGlobalFieldMapVersion i.e. its still set to 0
-    assert DefaultSegmentInfosWriter.FORMAT_4_0 <= format || (DefaultSegmentInfosWriter.FORMAT_4_0 > format && lastGlobalFieldMapVersion == 0); 
   }
 
   // Only non-null after prepareCommit has been called and
@@ -308,14 +292,6 @@ public final class SegmentInfos implemen
   private void write(Directory directory, Codec codec) throws IOException {
 
     String segmentFileName = getNextSegmentFileName();
-    final String globalFieldMapFile;
-    if (globalFieldNumberMap != null && globalFieldNumberMap.isDirty()) {
-      globalFieldMapFile = getGlobalFieldNumberName(++globalFieldMapVersion);
-      pendingMapVersion = writeGlobalFieldMap(globalFieldNumberMap, directory, globalFieldMapFile);
-    } else {
-      globalFieldMapFile = null;
-    }
-    
     
     // Always advance the generation on write:
     if (generation == -1) {
@@ -347,16 +323,6 @@ public final class SegmentInfos implemen
         } catch (Throwable t) {
           // Suppress so we keep throwing the original exception
         }
-        if (globalFieldMapFile != null) { // delete if written here
-          try {
-            // Try not to leave global field map in
-            // the index:
-            directory.deleteFile(globalFieldMapFile);
-          } catch (Throwable t) {
-            // Suppress so we keep throwing the original exception
-          }
-        }
-        pendingMapVersion = -1;
       }
     }
   }
@@ -767,8 +733,6 @@ public final class SegmentInfos implemen
   void updateGeneration(SegmentInfos other) {
     lastGeneration = other.lastGeneration;
     generation = other.generation;
-    lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion;
-    globalFieldMapVersion = other.globalFieldMapVersion;
   }
 
   final void rollbackCommit(Directory dir) throws IOException {
@@ -792,16 +756,6 @@ public final class SegmentInfos implemen
         // in our caller
       }
       pendingSegnOutput = null;
-      if (pendingMapVersion != -1) {
-        try {
-          final String fieldMapName = getGlobalFieldNumberName(globalFieldMapVersion--);
-          dir.deleteFile(fieldMapName);
-        } catch (Throwable t) {
-          // Suppress so we keep throwing the original exception
-          // in our caller
-        }
-        pendingMapVersion = -1;
-      }
     }
   }
 
@@ -820,44 +774,6 @@ public final class SegmentInfos implemen
       throw new IllegalStateException("prepareCommit was already called");
     write(dir, codec);
   }
-  
-  private final long writeGlobalFieldMap(FieldNumberBiMap map, Directory dir, String name) throws IOException {
-    final IndexOutput output = dir.createOutput(name, IOContext.READONCE);
-    boolean success = false;
-    long version;
-    try {
-      version = map.write(output);
-      success = true;
-    } finally {
-      try {
-        output.close();
-      } catch (Throwable t) {
-        // throw orig excp
-      }
-      if (!success) {
-        try {
-          dir.deleteFile(name);
-        } catch (Throwable t) {
-          // throw orig excp
-        }
-      } else {
-        // we must sync here explicitly since during a commit
-        // IW will not sync the global field map. 
-        dir.sync(Collections.singleton(name));
-      }
-    }
-    return version;
-  }
-  
-  private void readGlobalFieldMap(FieldNumberBiMap map, Directory dir) throws IOException {
-    final String name = getGlobalFieldNumberName(lastGlobalFieldMapVersion);
-    final IndexInput input = dir.openInput(name, IOContext.READONCE);
-    try {
-      map.read(input);
-    } finally {
-      input.close();
-    }
-  }
 
   /** Returns all file names referenced by SegmentInfo
    *  instances matching the provided Directory (ie files
@@ -876,9 +792,6 @@ public final class SegmentInfos implemen
         files.add(segmentFileName);
       }
     }
-    if (lastGlobalFieldMapVersion > 0) {
-      files.add(getGlobalFieldNumberName(lastGlobalFieldMapVersion));
-    }
     final int size = size();
     for(int i=0;i<size;i++) {
       final SegmentInfo info = info(i);
@@ -929,17 +842,6 @@ public final class SegmentInfos implemen
     }
 
     lastGeneration = generation;
-    if (pendingMapVersion != -1) {
-      /*
-       * TODO is it possible that the commit does not succeed here? if another
-       * commit happens at the same time and we lost the race between the
-       * prepareCommit and finishCommit the latest version is already
-       * incremented.
-       */
-      globalFieldNumberMap.commitLastVersion(pendingMapVersion);
-      pendingMapVersion = -1;
-      lastGlobalFieldMapVersion = globalFieldMapVersion;
-    }
 
     try {
       IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN, IOContext.READONCE);
@@ -1003,7 +905,6 @@ public final class SegmentInfos implemen
   void replace(SegmentInfos other) {
     rollbackSegmentInfos(other.asList());
     lastGeneration = other.lastGeneration;
-    lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion;
     format = other.format;
   }
 
@@ -1027,47 +928,24 @@ public final class SegmentInfos implemen
    * Loads or returns the already loaded the global field number map for this {@link SegmentInfos}.
    * If this {@link SegmentInfos} has no global field number map the returned instance is empty
    */
-  FieldNumberBiMap getOrLoadGlobalFieldNumberMap(Directory dir) throws IOException {
+  FieldNumberBiMap getOrLoadGlobalFieldNumberMap() throws IOException {
     if (globalFieldNumberMap != null) {
       return globalFieldNumberMap;
     }
     final FieldNumberBiMap map  = new FieldNumberBiMap();
     
-    if (lastGlobalFieldMapVersion > 0) {
-      // if we don't have a global map or this is a SI from a earlier version we just return the empty map;
-      readGlobalFieldMap(map, dir);
-    }
     if (size() > 0) {
-      if (format > DefaultSegmentInfosWriter.FORMAT_4_0) {
-        assert lastGlobalFieldMapVersion == 0;
-        // build the map up if we open a pre 4.0 index
-        for (SegmentInfo info : this) {
-          final FieldInfos segFieldInfos = info.getFieldInfos();
-          for (FieldInfo fi : segFieldInfos) {
-            map.addOrGet(fi.name, fi.number);
-          }
+      // build the map up
+      for (SegmentInfo info : this) {
+        final FieldInfos segFieldInfos = info.getFieldInfos();
+        for (FieldInfo fi : segFieldInfos) {
+          map.addOrGet(fi.name, fi.number);
         }
       }
     }
     return globalFieldNumberMap = map;
   }
 
-  /**
-   * Called by {@link SegmentInfosReader} when reading the global field map version
-   */
-  public void setGlobalFieldMapVersion(long version) {
-    lastGlobalFieldMapVersion = globalFieldMapVersion = version;
-  }
-
-  public long getGlobalFieldMapVersion() {
-    return globalFieldMapVersion;
-  }
-  
-  // for testing
-  long getLastGlobalFieldMapVersion() {
-    return lastGlobalFieldMapVersion;
-  }
-  
   /** applies all changes caused by committing a merge to this SegmentInfos */
   void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) {
     final Set<SegmentInfo> mergedAway = new HashSet<SegmentInfo>(merge.segments);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Thu Dec  1 12:12:36 2011
@@ -49,9 +49,6 @@ public class DefaultSegmentInfosReader e
     infos.version = input.readLong(); // read version
     infos.counter = input.readInt(); // read counter
     final int format = infos.getFormat();
-    if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
-      infos.setGlobalFieldMapVersion(input.readLong());
-    }
     for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
       SegmentInfo si = readSegmentInfo(directory, format, input);
       if (si.getVersion() == null) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Thu Dec  1 12:12:36 2011
@@ -68,7 +68,6 @@ public class DefaultSegmentInfosWriter e
       out.writeString(codecID); // write codecID
       out.writeLong(infos.version);
       out.writeInt(infos.counter); // write counter
-      out.writeLong(infos.getGlobalFieldMapVersion());
       out.writeInt(infos.size()); // write infos
       for (SegmentInfo si : infos) {
         writeInfo(out, si);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java Thu Dec  1 12:12:36 2011
@@ -57,10 +57,6 @@ public class SimpleTextSegmentInfosReade
     infos.counter = Integer.parseInt(readString(COUNTER.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert StringHelper.startsWith(scratch, FNX_VERSION);
-    infos.setGlobalFieldMapVersion(Long.parseLong(readString(FNX_VERSION.length, scratch)));
-    
-    SimpleTextUtil.readLine(input, scratch);
     assert StringHelper.startsWith(scratch, NUM_USERDATA);
     int numUserData = Integer.parseInt(readString(NUM_USERDATA.length, scratch));
     infos.userData = new HashMap<String,String>();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java Thu Dec  1 12:12:36 2011
@@ -43,7 +43,6 @@ public class SimpleTextSegmentInfosWrite
 
   final static BytesRef VERSION             = new BytesRef("version ");
   final static BytesRef COUNTER             = new BytesRef("counter ");
-  final static BytesRef FNX_VERSION         = new BytesRef("global field map version ");
   final static BytesRef NUM_USERDATA        = new BytesRef("user data entries ");
   final static BytesRef USERDATA_KEY        = new BytesRef("  key ");
   final static BytesRef USERDATA_VALUE      = new BytesRef("  value ");
@@ -87,11 +86,6 @@ public class SimpleTextSegmentInfosWrite
       SimpleTextUtil.write(out, COUNTER);
       SimpleTextUtil.write(out, Integer.toString(infos.counter), scratch);
       SimpleTextUtil.writeNewline(out);
-      
-      // global field map version
-      SimpleTextUtil.write(out, FNX_VERSION);
-      SimpleTextUtil.write(out, Long.toString(infos.getGlobalFieldMapVersion()), scratch);
-      SimpleTextUtil.writeNewline(out);
 
       // user data
       int numUserDataEntries = infos.getUserData() == null ? 0 : infos.getUserData().size();

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Thu Dec  1 12:12:36 2011
@@ -1082,7 +1082,7 @@ public class TestAddIndexes extends Luce
     w3.close();
     // we should now see segments_X,
     // segments.gen,_Y.cfs,_Y.cfe, _Z.fnx
-    assertEquals("Only one compound segment should exist, but got: " + Arrays.toString(dir.listAll()), 5, dir.listAll().length);
+    assertEquals("Only one compound segment should exist, but got: " + Arrays.toString(dir.listAll()), 4, dir.listAll().length);
     dir.close();
   }
   

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Thu Dec  1 12:12:36 2011
@@ -583,8 +583,7 @@ public class TestBackwardsCompatibility 
                                "_0_1.del",
                                "_0_1.s" + contentFieldIndex,
                                "segments_2",
-                               "segments.gen",
-                               "_1.fnx"};
+                               "segments.gen"};
 
       String[] actual = dir.listAll();
       Arrays.sort(expected);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java Thu Dec  1 12:12:36 2011
@@ -179,7 +179,6 @@ public class TestConsistentFieldNumbers 
         FieldInfos fis1 = sis.info(0).getFieldInfos();
         assertEquals("f1", fis1.fieldInfo(0).name);
         assertEquals("f2", fis1.fieldInfo(1).name);
-        assertTrue(dir.fileExists("_1.fnx"));
       }
       
 
@@ -203,8 +202,6 @@ public class TestConsistentFieldNumbers 
         assertEquals("f1", fis2.fieldInfo(0).name);
         assertNull(fis2.fieldInfo(1));
         assertEquals("f3", fis2.fieldInfo(2).name);
-        assertFalse(dir.fileExists("_1.fnx"));
-        assertTrue(dir.fileExists("_2.fnx"));
       }
 
       {
@@ -232,9 +229,6 @@ public class TestConsistentFieldNumbers 
         assertEquals("f1", fis3.fieldInfo(0).name);
         assertEquals("f2", fis3.fieldInfo(1).name);
         assertEquals("f3", fis3.fieldInfo(2).name);
-        assertFalse(dir.fileExists("_1.fnx"));
-        assertTrue(dir.fileExists("_2.fnx"));
-        assertFalse(dir.fileExists("_3.fnx"));
       }
 
       {
@@ -262,9 +256,6 @@ public class TestConsistentFieldNumbers 
       assertEquals("f1", fis1.fieldInfo(0).name);
       assertEquals("f2", fis1.fieldInfo(1).name);
       assertEquals("f3", fis1.fieldInfo(2).name);
-      assertFalse(dir.fileExists("_1.fnx"));
-      assertTrue(dir.fileExists("_2.fnx"));
-      assertFalse(dir.fileExists("_3.fnx"));
       dir.close();
     }
   }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1209047&r1=1209046&r2=1209047&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Thu Dec  1 12:12:36 2011
@@ -916,10 +916,8 @@ public class TestIndexWriterExceptions e
       } catch (RuntimeException re) {
         // Expected
       }
-      assertTrue(dir.fileExists("_1.fnx"));
       assertTrue(failure.failOnCommit && failure.failOnDeleteFile);
       w.rollback();
-      assertFalse(dir.fileExists("_1.fnx"));
       assertEquals(0, dir.listAll().length);
       dir.close();
     }