You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2011/05/29 11:30:31 UTC

svn commit: r1128845 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/backwards/ lucene/src/java/org/apache/lucene/store/ lucene/src/java/org/apache/lucene/util/ lucene/src/test-framework/org/apache/lucene/search/ lucene/src/test-framework/org/ap...

Author: mikemccand
Date: Sun May 29 09:30:31 2011
New Revision: 1128845

URL: http://svn.apache.org/viewvc?rev=1128845&view=rev
Log:
LUCENE-2834: don't use MD5 to make lock id for foreign lock file (it spawns thread on OS X)

Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/backwards/   (props changed)
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Constants.java
    lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java
    lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestMultiMMap.java
    lucene/dev/branches/branch_3x/solr/   (props changed)

Modified: lucene/dev/branches/branch_3x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/CHANGES.txt?rev=1128845&r1=1128844&r2=1128845&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/CHANGES.txt Sun May 29 09:30:31 2011
@@ -2,7 +2,12 @@ Lucene Change Log
 
 ======================= Lucene 3.x (not yet released) ================
 
-(No changes)
+Changes in runtime behavior
+
+* LUCENE-2834: the hash used to compute the lock file name when the
+  lock file is not stored in the index has changed.  This means you
+  will see a different lucene-XXX-write.lock in your lock directory.
+  (Robert Muir, Uwe Schindler, Mike McCandless)
 
 ======================= Lucene 3.2.0 =======================
 

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=1128845&r1=1128844&r2=1128845&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/store/FSDirectory.java Sun May 29 09:30:31 2011
@@ -22,8 +22,6 @@ import java.io.FileNotFoundException;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.RandomAccessFile;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
 
 import java.util.Collection;
 import static java.util.Collections.synchronizedSet;
@@ -113,15 +111,6 @@ import org.apache.lucene.util.Constants;
  * @see Directory
  */
 public abstract class FSDirectory extends Directory {
-  private final static MessageDigest DIGESTER;
-
-  static {
-    try {
-      DIGESTER = MessageDigest.getInstance("MD5");
-    } catch (NoSuchAlgorithmException e) {
-        throw new RuntimeException(e.toString(), e);
-    }
-  }
 
   /**
    * Default read chunk size.  This is a conditional default: on 32bit JVMs, it defaults to 100 MB.  On 64bit JVMs, it's
@@ -356,12 +345,6 @@ public abstract class FSDirectory extend
     return openInput(name, BufferedIndexInput.BUFFER_SIZE);
   }
 
-  /**
-   * So we can do some byte-to-hexchar conversion below
-   */
-  private static final char[] HEX_DIGITS =
-  {'0','1','2','3','4','5','6','7','8','9','a','b','c','d','e','f'};
-  
   @Override
   public String getLockID() {
     ensureOpen();
@@ -372,19 +355,12 @@ public abstract class FSDirectory extend
       throw new RuntimeException(e.toString(), e);
     }
 
-    byte digest[];
-    synchronized (DIGESTER) {
-      digest = DIGESTER.digest(dirName.getBytes());
-    }
-    StringBuilder buf = new StringBuilder();
-    buf.append("lucene-");
-    for (int i = 0; i < digest.length; i++) {
-      int b = digest[i];
-      buf.append(HEX_DIGITS[(b >> 4) & 0xf]);
-      buf.append(HEX_DIGITS[b & 0xf]);
+    int digest = 0;
+    for(int charIDX=0;charIDX<dirName.length();charIDX++) {
+      final char ch = dirName.charAt(charIDX);
+      digest = 31 * digest + ch;
     }
-
-    return buf.toString();
+    return "lucene-" + Integer.toHexString(digest);
   }
 
   /** Closes the store to future operations. */

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Constants.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Constants.java?rev=1128845&r1=1128844&r2=1128845&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Constants.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/util/Constants.java Sun May 29 09:30:31 2011
@@ -43,6 +43,8 @@ public final class Constants {
   public static final boolean WINDOWS = OS_NAME.startsWith("Windows");
   /** True iff running on SunOS. */
   public static final boolean SUN_OS = OS_NAME.startsWith("SunOS");
+  /** True iff running on Mac OS X */
+  public static final boolean MAC_OS_X = OS_NAME.startsWith("Mac OS X");
 
   public static final String OS_ARCH = System.getProperty("os.arch");
   public static final String OS_VERSION = System.getProperty("os.version");

Modified: lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java?rev=1128845&r1=1128844&r2=1128845&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java Sun May 29 09:30:31 2011
@@ -1,5 +1,22 @@
 package org.apache.lucene.search;
 
+/**
+ * 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.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -22,21 +39,6 @@ import org.apache.lucene.util._TestUtil;
 
 import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
 
-/**
- * Copyright 2005 Apache Software Foundation
- *
- * Licensed 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.
- */
 
 
 

Modified: lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1128845&r1=1128844&r2=1128845&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Sun May 29 09:30:31 2011
@@ -1010,7 +1010,7 @@ public abstract class LuceneTestCase ext
       final Class<? extends Directory> clazz = Class.forName(clazzName).asSubclass(Directory.class);
       // If it is a FSDirectory type, try its ctor(File)
       if (FSDirectory.class.isAssignableFrom(clazz)) {
-        final File tmpFile = File.createTempFile("test", "tmp", TEMP_DIR);
+        final File tmpFile = _TestUtil.createTempFile("test", "tmp", TEMP_DIR);
         tmpFile.delete();
         tmpFile.mkdir();
         registerTempFile(tmpFile);

Modified: lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java?rev=1128845&r1=1128844&r2=1128845&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java Sun May 29 09:30:31 2011
@@ -364,4 +364,51 @@ public class _TestUtil {
       throw new RuntimeException(e);
     }
   }
+  
+  /** 
+   * insecure, fast version of File.createTempFile
+   * uses Random instead of SecureRandom.
+   */
+  public static File createTempFile(String prefix, String suffix, File directory)
+      throws IOException {
+    // Force a prefix null check first
+    if (prefix.length() < 3) {
+      throw new IllegalArgumentException("prefix must be 3");
+    }
+    String newSuffix = suffix == null ? ".tmp" : suffix;
+    File result;
+    do {
+      result = genTempFile(prefix, newSuffix, directory);
+    } while (!result.createNewFile());
+    return result;
+  }
+
+  /* Temp file counter */
+  private static int counter = 0;
+
+  /* identify for differnt VM processes */
+  private static int counterBase = 0;
+
+  private static class TempFileLocker {};
+  private static TempFileLocker tempFileLocker = new TempFileLocker();
+
+  private static File genTempFile(String prefix, String suffix, File directory) {
+    int identify = 0;
+
+    synchronized (tempFileLocker) {
+      if (counter == 0) {
+        int newInt = new Random().nextInt();
+        counter = ((newInt / 65535) & 0xFFFF) + 0x2710;
+        counterBase = counter;
+      }
+      identify = counter++;
+    }
+
+    StringBuilder newName = new StringBuilder();
+    newName.append(prefix);
+    newName.append(counterBase);
+    newName.append(identify);
+    newName.append(suffix);
+    return new File(directory, newName.toString());
+  }
 }

Modified: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1128845&r1=1128844&r2=1128845&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Sun May 29 09:30:31 2011
@@ -86,7 +86,7 @@ public class TestBufferedIndexInput exte
     // NOTE: this does only test the chunked reads and NOT if the Bug is triggered.
     //final int tmpFileSize = 1024 * 1024 * 5;
     final int inputBufferSize = 128;
-    File tmpInputFile = File.createTempFile("IndexInput", "tmpFile");
+    File tmpInputFile = _TestUtil.createTempFile("IndexInput", "tmpFile", TEMP_DIR);
     tmpInputFile.deleteOnExit();
     writeBytes(tmpInputFile, TEST_FILE_LENGTH);
 

Modified: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestMultiMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestMultiMMap.java?rev=1128845&r1=1128844&r2=1128845&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestMultiMMap.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/store/TestMultiMMap.java Sun May 29 09:30:31 2011
@@ -51,7 +51,7 @@ public class TestMultiMMap extends Lucen
   }
   
   private void assertChunking(Random random, int chunkSize) throws Exception {
-    File path = File.createTempFile("mmap" + chunkSize, "tmp", workDir);
+    File path = _TestUtil.createTempFile("mmap" + chunkSize, "tmp", workDir);
     path.delete();
     path.mkdirs();
     MMapDirectory dir = new MMapDirectory(path);