You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pylucene-commits@lucene.apache.org by va...@apache.org on 2016/08/27 15:42:28 UTC

svn commit: r1758054 - in /lucene/pylucene/trunk: ./ java/org/apache/pylucene/store/ test/

Author: vajda
Date: Sat Aug 27 15:42:27 2016
New Revision: 1758054

URL: http://svn.apache.org/viewvc?rev=1758054&view=rev
Log:
test_PythonDirectory.py passes

Modified:
    lucene/pylucene/trunk/extensions.xml
    lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonDirectory.java
    lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonIndexOutput.java
    lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLock.java
    lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLockFactory.java
    lucene/pylucene/trunk/test/test_PythonDirectory.py

Modified: lucene/pylucene/trunk/extensions.xml
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/extensions.xml?rev=1758054&r1=1758053&r2=1758054&view=diff
==============================================================================
--- lucene/pylucene/trunk/extensions.xml (original)
+++ lucene/pylucene/trunk/extensions.xml Sat Aug 27 15:42:27 2016
@@ -25,6 +25,8 @@
            destdir="${classes.dir}" classpathref="classpath" />
     <javac srcdir="java/org/apache/pylucene/index" 
            destdir="${classes.dir}" classpathref="classpath" />
+    <javac srcdir="java/org/apache/pylucene/store" 
+           destdir="${classes.dir}" classpathref="classpath" />
   </target>
 
   <target name="jar" depends="compile">

Modified: lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonDirectory.java
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonDirectory.java?rev=1758054&r1=1758053&r2=1758054&view=diff
==============================================================================
--- lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonDirectory.java (original)
+++ lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonDirectory.java Sat Aug 27 15:42:27 2016
@@ -50,7 +50,7 @@ public class PythonDirectory extends Dir
     }
 
     public void sync(Collection<String> names)
-        throws IOException 
+        throws IOException
     {
         for (String name : names)
             sync(name);
@@ -62,26 +62,22 @@ public class PythonDirectory extends Dir
         throws IOException;
     public native IndexOutput createOutput(String name, IOContext context)
         throws IOException;
-    public native void deleteFile(String name)
+    public native IndexOutput createTempOutput(String prefix, String suffix,
+                                               IOContext context)
         throws IOException;
-    public native boolean fileExists(String name)
+    public native void deleteFile(String name)
         throws IOException;
     public native long fileLength(String name)
         throws IOException;
-    public native long fileModified(String name)
-        throws IOException;
     public native String[] listAll()
         throws IOException;
     public native IndexInput openInput(String name, IOContext context)
         throws IOException;
-    public native void touchFile(String name)
-        throws IOException;
-    public native void sync(String name) 
+    public native void sync(String name)
         throws IOException;
-    public native LockFactory getLockFactory();
-    public native void setLockFactory(LockFactory lockFactory)
+    public native void rename(String source, String dest)
         throws IOException;
-    public native void clearLock(String name)
+    public native void syncMetaData()
         throws IOException;
-    public native Lock makeLock(String name);
+    public native Lock obtainLock(String name);
 }

Modified: lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonIndexOutput.java?rev=1758054&r1=1758053&r2=1758054&view=diff
==============================================================================
--- lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonIndexOutput.java (original)
+++ lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonIndexOutput.java Sat Aug 27 15:42:27 2016
@@ -23,8 +23,9 @@ public class PythonIndexOutput extends I
 
     private long pythonObject;
 
-    public PythonIndexOutput()
+    public PythonIndexOutput(String resourceDescription, String name)
     {
+        super(resourceDescription, name);
     }
 
     public void pythonExtension(long pythonObject)

Modified: lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLock.java
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLock.java?rev=1758054&r1=1758053&r2=1758054&view=diff
==============================================================================
--- lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLock.java (original)
+++ lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLock.java Sat Aug 27 15:42:27 2016
@@ -15,6 +15,7 @@
 
 package org.apache.pylucene.store;
 
+import java.io.IOException;
 import org.apache.lucene.store.Lock;
 
 
@@ -42,8 +43,8 @@ public class PythonLock extends Lock {
     }
 
     public native void pythonDecRef();
-    public native boolean isLocked();
-    public native boolean obtain();
-    public native void release();
+
+    public native void ensureValid()
+        throws IOException;
     public native void close();
 }

Modified: lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLockFactory.java?rev=1758054&r1=1758053&r2=1758054&view=diff
==============================================================================
--- lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLockFactory.java (original)
+++ lucene/pylucene/trunk/java/org/apache/pylucene/store/PythonLockFactory.java Sat Aug 27 15:42:27 2016
@@ -17,6 +17,7 @@ package org.apache.pylucene.store;
 
 import java.io.IOException;
 
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockFactory;
 
@@ -46,7 +47,6 @@ public class PythonLockFactory extends L
 
     public native void pythonDecRef();
 
-    public native Lock makeLock(String lockName);
-    public native void clearLock(String lockName)
+    public native Lock obtainLock(Directory dir, String lockName)
         throws IOException;
 }

Modified: lucene/pylucene/trunk/test/test_PythonDirectory.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/test/test_PythonDirectory.py?rev=1758054&r1=1758053&r2=1758054&view=diff
==============================================================================
--- lucene/pylucene/trunk/test/test_PythonDirectory.py (original)
+++ lucene/pylucene/trunk/test/test_PythonDirectory.py Sat Aug 27 15:42:27 2016
@@ -14,7 +14,7 @@
 
 import sys, lucene, unittest
 import os, shutil
-import test_PyLucene 
+import test_PyLucene
 from binascii import crc32
 from threading import RLock
 from lucene import JavaError, JArray
@@ -28,7 +28,7 @@ from org.apache.pylucene.store import \
 """
 The Directory Implementation here is for testing purposes only, not meant
 as an example of writing one, the implementation here suffers from a lack
-of safety when dealing with concurrent modifications as it does away with 
+of safety when dealing with concurrent modifications as it does away with
 the file locking in the default lucene fsdirectory implementation.
 """
 
@@ -43,65 +43,17 @@ class DebugWrapper(object):
         print self.obj.__class__.__name__, self.obj.name, name
         sys.stdout.flush()
         return getattr(self.obj, name)
-        
+
 class DebugFactory(object):
-    
+
     def __init__(self, klass):
         self.klass = klass
-        
+
     def __call__(self, *args, **kw):
         instance = self.klass(*args, **kw)
         return DebugWrapper(instance)
 
 
-class PythonDirLock(PythonLock):
-    # only safe for a single process
-    
-    def __init__(self, name, path, lock):
-        super(PythonDirLock, self).__init__()
-
-        self.name = name
-        self.lock_file = path
-        self.lock = lock
-
-    def isLocked(self):
-        return self.lock.locked()
-
-    def obtain(self):
-        return self.lock.acquire()
-
-    def release(self):
-        return self.lock.release()
-
-    def close(self):
-        if hasattr(self.lock, 'close'):
-            self.lock.close()
-
-
-class PythonDirLockFactory(PythonLockFactory):
-
-    def __init__(self, path):
-        super(PythonDirLockFactory, self).__init__()
-        
-        self.path = path
-        self._locks = {}
-
-    def makeLock(self, name):
-
-        lock = self._locks.get(name)
-        if lock is None:
-            lock = PythonDirLock(name, os.path.join(self.path, name), RLock())
-            self._locks[name] = lock
-
-        return lock
-
-    def clearLock(self, name):
-
-        lock = self._locks.pop(name, None)
-        if lock is not None:
-            lock.release()
-
-
 class PythonFileStreamInput(PythonIndexInput):
 
     def __init__(self, name, fh, size, clone=False):
@@ -137,8 +89,7 @@ class PythonFileStreamInput(PythonIndexI
 class PythonFileStreamOutput(PythonIndexOutput):
 
     def __init__(self, name, fh):
-        super(PythonFileStreamOutput, self).__init__()
-        self.name = name
+        super(PythonFileStreamOutput, self).__init__("python", name)
         self.fh = fh
         self.isOpen = True
         self._length = 0
@@ -186,11 +137,43 @@ class PythonFileDirectory(PythonDirector
     def __init__(self, path):
         super(PythonFileDirectory, self).__init__()
 
-        self._lockFactory = PythonDirLockFactory(path)
+        class _lock_factory(PythonLockFactory):
+
+            def __init__(_self):
+                super(_lock_factory, _self).__init__()
+                _self._locks = {}
+
+            def obtainLock(_self, directory, name):
+
+                # only safe for a single process
+                class _lock(PythonLock):
+
+                    def __init__(__self, path):
+                        super(_lock, __self).__init__()
+
+                        __self.lock_file = path
+                        __self.lock = RLock()
+
+                    def ensureValid(__self):
+                        __self.lock.acquire()
+
+                    def close(__self):
+                        if hasattr(__self.lock, 'close'):
+                            __self.lock.close()
+
+                lock = _self._locks.get(name)
+                if lock is None:
+                    lock = _lock(os.path.join(directory.name, name))
+                    _self._locks[name] = lock
+
+                return lock
+
+        self._lockFactory = _lock_factory()
         self.name = path
         assert os.path.isdir(path)
         self.path = path
         self._streams = []
+        self.temp_count = 0
 
     def close(self):
         for stream in self._streams:
@@ -204,26 +187,29 @@ class PythonFileDirectory(PythonDirector
         self._streams.append(stream)
         return stream
 
-    def deleteFile(self, name):
-        if self.fileExists(name):
-            os.unlink(os.path.join(self.path, name))
+    def createTempOutput(self, prefix, suffix, context):
+        self.temp_count += 1
+        name = "%s_%d_%s.tmp" %(prefix, self.temp_count, suffix)
+        return self.createOutput(name, context)
 
-    def fileExists(self, name):
-        return os.path.exists(os.path.join(self.path, name))
+    def deleteFile(self, name):
+        os.unlink(os.path.join(self.path, name))
 
     def fileLength(self, name):
         file_path = os.path.join(self.path, name)
         return long(os.path.getsize(file_path))
 
-    def fileModified(self, name):
-        file_path = os.path.join(self.path, name)
-        return os.path.getmtime(file_path)
-
     def listAll(self):
         return os.listdir(self.path)
 
     def sync(self, name):
         pass
+    def syncMetaData(self):
+        pass
+
+    def rename(self, source, dest):
+        shutil.move(os.path.join(self.path, source),
+                    os.path.join(self.path, dest))
 
     def openInput(self, name, bufferSize=0):
         file_path = os.path.join(self.path, name)
@@ -235,21 +221,8 @@ class PythonFileDirectory(PythonDirector
         self._streams.append(stream)
         return stream
 
-    def touchFile(self, name):
-        file_path = os.path.join(self.path, name)
-        os.utime(file_path, None)
-
-    def setLockFactory(self, lockFactory):
-        pass
-
-    def getLockFactory(self):
-        return None
-
-    def clearLock(self, name):
-        self._lockFactory.clearLock(name)
-
-    def makeLock(self, name):
-        return self._lockFactory.makeLock(name)
+    def obtainLock(self, name):
+        return self._lockFactory.obtainLock(self, name)
 
 
 if DEBUG:
@@ -287,7 +260,7 @@ class PythonDirectoryTests(unittest.Test
             print "indexing ", i
             sys.stdout.flush()
             self.test_indexDocument()
-                       
+
 
 if __name__ == "__main__":
     env = lucene.initVM(vmargs=['-Djava.awt.headless=true'])