You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucy.apache.org by ma...@apache.org on 2009/12/07 21:12:32 UTC

svn commit: r888114 - in /lucene/lucy/trunk: core/Lucy/Store/ core/Lucy/Test/Store/ perl/lib/ perl/lib/Lucy/ perl/lib/Lucy/Store/ perl/t/core/ perl/xs/Lucy/Store/

Author: marvin
Date: Mon Dec  7 20:12:30 2009
New Revision: 888114

URL: http://svn.apache.org/viewvc?rev=888114&view=rev
Log:
Commit LUCY-85, adding FSFolder.  Also, change semantics of Folder_Rename() to
reflext ambiguity of C standard for rename() when target file exists.

Added:
    lucene/lucy/trunk/core/Lucy/Store/FSFolder.bp   (with props)
    lucene/lucy/trunk/core/Lucy/Store/FSFolder.c   (with props)
    lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.bp   (with props)
    lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.c   (with props)
    lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.bp   (with props)
    lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.c   (with props)
    lucene/lucy/trunk/perl/lib/Lucy/Store/FSFolder.pm   (with props)
    lucene/lucy/trunk/perl/t/core/103-fs_folder.t   (with props)
    lucene/lucy/trunk/perl/xs/Lucy/Store/
    lucene/lucy/trunk/perl/xs/Lucy/Store/FSFolder.c   (with props)
Modified:
    lucene/lucy/trunk/core/Lucy/Store/Folder.bp
    lucene/lucy/trunk/perl/lib/Lucy.pm
    lucene/lucy/trunk/perl/lib/Lucy/Test.pm

Added: lucene/lucy/trunk/core/Lucy/Store/FSFolder.bp
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/core/Lucy/Store/FSFolder.bp?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/core/Lucy/Store/FSFolder.bp (added)
+++ lucene/lucy/trunk/core/Lucy/Store/FSFolder.bp Mon Dec  7 20:12:30 2009
@@ -0,0 +1,85 @@
+parcel Lucy;
+
+/** File System implementation of Folder.
+ *
+ * Implementation of Lucy::Store::Folder using a single file system 
+ * directory and multiple files.
+ */
+
+class Lucy::Store::FSFolder extends Lucy::Store::Folder {
+
+    inert incremented FSFolder*
+    new(const CharBuf *path);
+
+    /**
+     * @param path Location of the index. If the specified directory does
+     * not exist already, it will NOT be created, in order to prevent
+     * misconfigured read applications from spawning bogus files -- so it may
+     * be necessary to create the directory yourself.
+     */
+    public inert FSFolder*
+    init(FSFolder *self, const CharBuf *path);
+
+    /** Attempt to create the directory specified by <code>path</code>.
+     */
+    public void
+    Initialize(FSFolder *self);
+
+    /** Verify that <code>path</code> is a directory.  TODO: check
+     * permissions.
+     */
+    public bool_t
+    Check(FSFolder *self);
+
+    public void
+    Close(FSFolder *self);
+
+    incremented FileHandle*
+    Local_Open_FileHandle(FSFolder *self, const CharBuf *name, 
+                          u32_t flags);
+
+    incremented DirHandle*
+    Local_Open_Dir(FSFolder *self);
+
+    bool_t 
+    Local_MkDir(FSFolder *self, const CharBuf *name);
+
+    bool_t
+    Local_Exists(FSFolder *self, const CharBuf *name);
+
+    bool_t
+    Local_Is_Directory(FSFolder *self, const CharBuf *name);
+
+    Folder*
+    Local_Find_Folder(FSFolder *self, const CharBuf *name);
+
+    bool_t 
+    Local_Delete(FSFolder *self, const CharBuf *name);
+
+    public bool_t 
+    Rename(FSFolder *self, const CharBuf* from, const CharBuf *to);
+
+    public bool_t
+    Hard_Link(FSFolder *self, const CharBuf *from, const CharBuf *to);
+
+    /** Transform a relative path into an abolute path.
+     */
+    inert incremented CharBuf*
+    absolutify(const CharBuf *path);
+}
+
+/* Copyright 2009 The 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.
+ */
+

Propchange: lucene/lucy/trunk/core/Lucy/Store/FSFolder.bp
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/lucy/trunk/core/Lucy/Store/FSFolder.c
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/core/Lucy/Store/FSFolder.c?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/core/Lucy/Store/FSFolder.c (added)
+++ lucene/lucy/trunk/core/Lucy/Store/FSFolder.c Mon Dec  7 20:12:30 2009
@@ -0,0 +1,333 @@
+#define C_LUCY_FSFOLDER
+#include "Lucy/Util/ToolSet.h"
+
+#include <ctype.h>
+#include <errno.h>
+#include <fcntl.h>
+#include <stdio.h>
+#include <sys/stat.h>
+
+#ifdef CHY_HAS_SYS_TYPES_H
+  #include <sys/types.h>
+#endif
+
+/* For rmdir, (hard) link. */
+#ifdef CHY_HAS_UNISTD_H
+  #include <unistd.h>
+#endif
+
+/* For CreateHardLink. */
+#ifdef CHY_HAS_WINDOWS_H
+  #include <windows.h>
+#endif
+
+/* For mkdir, rmdir. */
+#ifdef CHY_HAS_DIRECT_H
+  #include <direct.h>
+#endif
+
+#include "Lucy/Store/FSFolder.h"
+#include "Lucy/Store/CompoundFileReader.h"
+#include "Lucy/Store/CompoundFileWriter.h"
+#include "Lucy/Store/FSDirHandle.h"
+#include "Lucy/Store/FSFileHandle.h"
+#include "Lucy/Store/InStream.h"
+#include "Lucy/Store/OutStream.h"
+#include "Lucy/Util/IndexFileNames.h"
+
+/* Return a new CharBuf containing a platform-specific absolute filepath. */
+static CharBuf*
+S_fullpath(FSFolder *self, const CharBuf *path);
+
+/* Return true if the supplied path is a directory. */
+static bool_t
+S_dir_ok(const CharBuf *path);
+
+/* Create a directory, or set Err_error and return false. */
+bool_t
+S_create_dir(const CharBuf *path);
+
+/* Return true unless the supplied path contains a slash. */
+bool_t
+S_is_local_entry(const CharBuf *path);
+
+FSFolder*
+FSFolder_new(const CharBuf *path) 
+{
+    FSFolder *self = (FSFolder*)VTable_Make_Obj(FSFOLDER);
+    return FSFolder_init(self, path);
+}
+
+FSFolder*
+FSFolder_init(FSFolder *self, const CharBuf *path)
+{
+    CharBuf *abs_path = FSFolder_absolutify(path);
+    Folder_init((Folder*)self, abs_path);
+    DECREF(abs_path);
+    return self;
+}
+
+void
+FSFolder_initialize(FSFolder *self)
+{
+    if (!S_dir_ok(self->path)) {
+        if (!S_create_dir(self->path)) {
+            RETHROW(INCREF(Err_get_error()));
+        }
+    }
+}
+
+bool_t
+FSFolder_check(FSFolder *self)
+{
+    return S_dir_ok(self->path);
+}
+
+FileHandle*
+FSFolder_local_open_filehandle(FSFolder *self, const CharBuf *name, u32_t flags)
+{
+    CharBuf      *fullpath = S_fullpath(self, name);
+    FSFileHandle *fh = FSFH_open(fullpath, flags);
+    if (!fh) {
+        ERR_ADD_FRAME(Err_get_error());
+    }
+    DECREF(fullpath);
+    return (FileHandle*)fh;
+}
+
+bool_t
+FSFolder_local_mkdir(FSFolder *self, const CharBuf *name)
+{
+    CharBuf *dir = S_fullpath(self, name);
+    bool_t result = S_create_dir(dir);
+    DECREF(dir);
+    if (!result) {
+        ERR_ADD_FRAME(Err_get_error());
+    }
+    return result;
+}
+
+DirHandle*
+FSFolder_local_open_dir(FSFolder *self)
+{
+    DirHandle *dh = (DirHandle*)FSDH_open(self->path);
+    if (!dh) { ERR_ADD_FRAME(Err_get_error()); }
+    return dh;
+}
+
+bool_t
+FSFolder_local_exists(FSFolder *self, const CharBuf *name)
+{
+    if (Hash_Fetch(self->entries, (Obj*)name)) {
+        return true;
+    }
+    else if (!S_is_local_entry(name)) {
+        return false;
+    }
+    else {
+        struct stat stat_buf;
+        CharBuf *fullpath = S_fullpath(self, name);
+        bool_t retval = false;
+        if (stat((char*)CB_Get_Ptr8(fullpath), &stat_buf) != -1)
+            retval = true;
+        DECREF(fullpath);
+        return retval;
+    }
+}
+
+bool_t
+FSFolder_local_is_directory(FSFolder *self, const CharBuf *name)
+{
+    /* Check for a cached object, then fall back to a system call. */
+    Obj *elem = Hash_Fetch(self->entries, (Obj*)name);
+    if (elem && Obj_Is_A(elem, FOLDER)) { 
+        return true; 
+    }
+    else {
+        CharBuf *fullpath = S_fullpath(self, name);
+        bool_t result = S_dir_ok(fullpath);
+        DECREF(fullpath);
+        return result;
+    }
+}
+
+bool_t
+FSFolder_rename(FSFolder *self, const CharBuf* from, const CharBuf *to)
+{
+    CharBuf *from_path = S_fullpath(self, from);
+    CharBuf *to_path   = S_fullpath(self, to);
+    bool_t   retval    = !rename((char*)CB_Get_Ptr8(from_path), 
+        (char*)CB_Get_Ptr8(to_path));
+    if (!retval) {
+        Err_set_error(Err_new(CB_newf("rename from '%o' to '%o' failed: %s",
+            from_path, to_path, strerror(errno))));
+    }
+    DECREF(to_path);
+    DECREF(from_path);
+    return retval;
+}
+
+bool_t
+FSFolder_hard_link(FSFolder *self, const CharBuf *from, 
+                   const CharBuf *to)
+{
+    CharBuf *from_path = S_fullpath(self, from);
+    CharBuf *to_path   = S_fullpath(self, to);
+    char    *from8     = (char*)CB_Get_Ptr8(from_path);
+    char    *to8       = (char*)CB_Get_Ptr8(to_path);
+#ifdef CHY_HAS_UNISTD_H
+    bool_t retval;
+    if (-1 == link(from8, to8)) {
+        retval = false;
+        Err_set_error(Err_new(CB_newf(
+            "hard link for new file '%o' from '%o' failed: %s",
+                to_path, from_path, strerror(errno))));
+    }
+    else {
+        retval = true;
+    }
+#elif defined(CHY_HAS_WINDOWS_H)
+    bool_t retval = !!CreateHardLink(to8, from8, NULL);
+    if (!retval) {
+        char *win_error = Err_win_error();
+        Err_set_error(Err_new(CB_newf(
+            "CreateHardLink for new file '%o' from '%o' failed: %s",
+                to_path, from_path, win_error)));
+        FREEMEM(win_error);
+    }
+#endif
+    DECREF(to_path);
+    DECREF(from_path);
+    return retval;
+}
+
+bool_t
+FSFolder_local_delete(FSFolder *self, const CharBuf *name)
+{
+    CharBuf *fullpath = S_fullpath(self, name);
+    char    *path_ptr = (char*)CB_Get_Ptr8(fullpath);
+#ifdef CHY_REMOVE_ZAPS_DIRS
+    bool_t result = !remove(path_ptr);
+#else 
+    bool_t result = !rmdir(path_ptr) || !remove(path_ptr); 
+#endif
+    DECREF(fullpath);
+    DECREF(Hash_Delete(self->entries, (Obj*)name));
+    return result;
+}
+
+void
+FSFolder_close(FSFolder *self)
+{
+    Hash_Clear(self->entries);
+}
+
+Folder*
+FSFolder_local_find_folder(FSFolder *self, const CharBuf *name)
+{
+    Folder *subfolder = NULL;
+    if (!name || !CB_Get_Size(name)) {
+        /* No entity can be identified by NULL or empty string. */
+        return NULL;
+    }
+    else if (!S_is_local_entry(name)) {
+        return NULL;
+    }
+    else if (NULL != (subfolder = (Folder*)Hash_Fetch(self->entries, (Obj*)name))) {
+        if (Obj_Is_A(subfolder, FOLDER)) {
+            return subfolder;
+        }
+        else {
+            return NULL;
+        }
+    }
+
+    {
+        CharBuf *fullpath = S_fullpath(self, name);
+        static ZombieCharBuf cfmeta_file = ZCB_LITERAL("cfmeta.json");
+        if (S_dir_ok(fullpath)) {
+            subfolder = (Folder*)FSFolder_new(fullpath);
+            if (!subfolder) {
+                THROW(ERR, "Failed to open FSFolder at '%o'", fullpath);
+            }
+            /* Try to open a CompoundFileReader. On failure, just use the
+             * existing folder. */
+            if (Folder_Local_Exists(subfolder, (CharBuf*)&cfmeta_file)) {
+                CompoundFileReader *cf_reader = CFReader_open(subfolder);
+                if (cf_reader) {
+                    DECREF(subfolder);
+                    subfolder = (Folder*)cf_reader;
+                }
+            }
+            Hash_Store(self->entries, (Obj*)name, (Obj*)subfolder);
+            DECREF(fullpath);
+            return subfolder;
+        }
+        else {
+            DECREF(fullpath);
+        }
+    }
+
+    return NULL;
+}
+
+static CharBuf*
+S_fullpath(FSFolder *self, const CharBuf *path)
+{
+    CharBuf *fullpath = CB_new(200);
+    CB_Cat(fullpath, self->path);
+    CB_Cat_Trusted_Str(fullpath, DIR_SEP, sizeof(DIR_SEP) - 1);
+    CB_Cat(fullpath, path);
+    if (DIR_SEP[0] != '/') {
+        CB_Swap_Chars(fullpath, '/', DIR_SEP[0]);
+    }
+    return fullpath;
+}
+
+static bool_t
+S_dir_ok(const CharBuf *path)
+{
+    struct stat stat_buf;
+    if (stat((char*)CB_Get_Ptr8(path), &stat_buf) != -1) {
+        if (stat_buf.st_mode & S_IFDIR) return true;
+    }
+    return false;
+}
+
+bool_t
+S_create_dir(const CharBuf *path)
+{
+    if (-1 == chy_makedir((char*)CB_Get_Ptr8(path), 0777)) {
+        Err_set_error(Err_new(CB_newf(
+            "Couldn't create directory '%o': %s", path, strerror(errno))));
+        return false;
+    }
+    return true;
+}
+
+bool_t
+S_is_local_entry(const CharBuf *path)
+{
+    ZombieCharBuf scratch = ZCB_make(path);
+    u32_t code_point;
+    while (0 != (code_point = ZCB_Nip_One(&scratch))) {
+        if (code_point == '/') { return false; }
+    }
+    return true;
+}
+
+/* Copyright 2009 The 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.
+ */
+

Propchange: lucene/lucy/trunk/core/Lucy/Store/FSFolder.c
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/lucy/trunk/core/Lucy/Store/Folder.bp
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/core/Lucy/Store/Folder.bp?rev=888114&r1=888113&r2=888114&view=diff
==============================================================================
--- lucene/lucy/trunk/core/Lucy/Store/Folder.bp (original)
+++ lucene/lucy/trunk/core/Lucy/Store/Folder.bp Mon Dec  7 20:12:30 2009
@@ -115,7 +115,8 @@
     Delete_Tree(Folder *self, const CharBuf *path);
 
     /** Rename a file or directory, or set Err_error and return false on
-     * failure.  If an entry exists at <code>to</code>, it will be removed.
+     * failure.  If an entry exists at <code>to</code>, the results are
+     * undefined.
      *
      * @param from The filepath prior to renaming.
      * @param to The filepath after renaming.

Added: lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.bp
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.bp?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.bp (added)
+++ lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.bp Mon Dec  7 20:12:30 2009
@@ -0,0 +1,22 @@
+parcel Lucy;
+
+inert class Lucy::Test::Store::TestFSFolder {
+    inert void
+    run_tests();
+}
+
+/* Copyright 2009 The 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.
+ */
+

Propchange: lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.bp
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.c
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.c?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.c (added)
+++ lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.c Mon Dec  7 20:12:30 2009
@@ -0,0 +1,90 @@
+#define C_LUCY_CHARBUF
+#include "Lucy/Util/ToolSet.h"
+
+/* mkdir, rmdir */
+#ifdef CHY_HAS_DIRECT_H
+  #include <direct.h>
+#endif
+
+/* rmdir */
+#ifdef CHY_HAS_UNISTD_H
+  #include <unistd.h>
+#endif
+
+/* mkdir, stat */
+#ifdef CHY_HAS_SYS_STAT_H
+  #include <sys/stat.h>
+#endif
+
+#include "Lucy/Test.h"
+#include "Lucy/Test/Store/TestFSFolder.h"
+#include "Lucy/Test/Store/TestFolderCommon.h"
+#include "Lucy/Store/FSFolder.h"
+
+static CharBuf test_dir_name = ZCB_LITERAL("_fsfolder_test");
+
+static Folder*
+S_set_up()
+{
+    FSFolder *folder = FSFolder_new(&test_dir_name);
+    rmdir("_fsfolder_test");
+    FSFolder_Initialize(folder);
+    if (!FSFolder_Check(folder)) {
+        RETHROW(INCREF(Err_get_error()));
+    }
+    return (Folder*)folder;
+}
+
+static void
+S_tear_down()
+{
+    struct stat stat_buf;
+    rmdir("_fsfolder_test");
+    if (stat("_fsfolder_test", &stat_buf) != -1) {
+        THROW(ERR, "Can't clean up directory _fsfolder_test");
+    }
+}
+
+static void
+test_Initialize_and_Check(TestBatch *batch)
+{
+    FSFolder *folder = FSFolder_new(&test_dir_name);
+    rmdir("_fsfolder_test");
+    ASSERT_FALSE(batch, Folder_Check(folder), 
+        "Check() returns false when folder dir doesn't exist");
+    Folder_Initialize(folder);
+    PASS(batch, "Initialize() concludes without incident");
+    ASSERT_TRUE(batch, Folder_Check(folder), 
+        "Initialize() created dir, and now Check() succeeds");
+    DECREF(folder);
+    S_tear_down();
+}
+
+void
+TestFSFolder_run_tests()
+{
+    u32_t num_tests = TestFolderCommon_num_tests() + 3;
+    TestBatch *batch = Test_new_batch("TestFSFolder", num_tests, NULL);
+
+    PLAN(batch);
+    test_Initialize_and_Check(batch);
+    TestFolderCommon_run_tests(batch, S_set_up, S_tear_down);
+
+    batch->destroy(batch);
+}
+
+/* Copyright 2009 The 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.
+ */
+

Propchange: lucene/lucy/trunk/core/Lucy/Test/Store/TestFSFolder.c
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.bp
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.bp?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.bp (added)
+++ lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.bp Mon Dec  7 20:12:30 2009
@@ -0,0 +1,38 @@
+parcel Lucy;
+
+__C__
+typedef lucy_Folder*
+lucy_TestFolderCommon_set_up_t(void);
+typedef void
+lucy_TestFolderCommon_tear_down_t(void);
+#ifdef LUCY_USE_SHORT_NAMES
+  #define TestFolderCommon_set_up_t    lucy_TestFolderCommon_set_up_t
+  #define TestFolderCommon_tear_down_t lucy_TestFolderCommon_tear_down_t
+#endif
+__END_C__
+
+inert class Lucy::Test::Store::TestFolderCommon {
+    inert u32_t
+    num_tests();
+
+    inert void
+    run_tests(void *test_batch, 
+              lucy_TestFolderCommon_set_up_t set_up, 
+              lucy_TestFolderCommon_tear_down_t tear_down);
+}
+
+/* Copyright 2009 The 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.
+ */
+

Propchange: lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.bp
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.c
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.c?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.c (added)
+++ lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.c Mon Dec  7 20:12:30 2009
@@ -0,0 +1,538 @@
+#define C_LUCY_CHARBUF
+#include "Lucy/Util/ToolSet.h"
+
+#include "Lucy/Test.h"
+#include "Lucy/Test/Store/TestFolderCommon.h"
+#include "Lucy/Store/Folder.h"
+#include "Lucy/Store/DirHandle.h"
+#include "Lucy/Store/FileHandle.h"
+#include "Lucy/Store/InStream.h"
+#include "Lucy/Store/OutStream.h"
+
+#define set_up_t    lucy_TestFolderCommon_set_up_t
+#define tear_down_t lucy_TestFolderCommon_tear_down_t
+
+static CharBuf test_index    = ZCB_LITERAL("test_index");
+static CharBuf foo           = ZCB_LITERAL("foo");
+static CharBuf bar           = ZCB_LITERAL("bar");
+static CharBuf baz           = ZCB_LITERAL("baz");
+static CharBuf boffo         = ZCB_LITERAL("boffo");
+static CharBuf banana        = ZCB_LITERAL("banana");
+static CharBuf foo_bar       = ZCB_LITERAL("foo/bar");
+static CharBuf foo_bar_baz   = ZCB_LITERAL("foo/bar/baz");
+static CharBuf foo_bar_boffo = ZCB_LITERAL("foo/bar/boffo");
+static CharBuf foo_boffo     = ZCB_LITERAL("foo/boffo");
+static CharBuf foo_foo       = ZCB_LITERAL("foo/foo");
+static CharBuf nope          = ZCB_LITERAL("nope");
+static CharBuf nope_nyet     = ZCB_LITERAL("nope/nyet");
+
+static void
+test_Local_Exists(TestBatch *batch, set_up_t set_up, tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    OutStream *outstream = Folder_Open_Out(folder, &boffo);
+    DECREF(outstream);
+    Folder_Local_MkDir(folder, &foo);
+    outstream = Folder_Open_Out(folder, &foo_boffo);
+    DECREF(outstream);
+
+    ASSERT_TRUE(batch, Folder_Local_Exists(folder, &boffo), 
+        "Local_Exists() returns true for file");
+    ASSERT_TRUE(batch, Folder_Local_Exists(folder, &foo), 
+        "Local_Exists() returns true for dir");
+    ASSERT_FALSE(batch, Folder_Local_Exists(folder, &foo_boffo), 
+        "Local_Exists() returns false for nested entry");
+    ASSERT_FALSE(batch, Folder_Local_Exists(folder, &bar), 
+        "Local_Exists() returns false for non-existent entry");
+
+    Folder_Delete(folder, &foo_boffo);
+    Folder_Delete(folder, &foo);
+    Folder_Delete(folder, &boffo);
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Local_Is_Directory(TestBatch *batch, set_up_t set_up, 
+                        tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    OutStream *outstream = Folder_Open_Out(folder, &boffo);
+    DECREF(outstream);
+    Folder_Local_MkDir(folder, &foo);
+
+    ASSERT_FALSE(batch, Folder_Local_Is_Directory(folder, &boffo), 
+        "Local_Is_Directory() returns false for file");
+    ASSERT_TRUE(batch, Folder_Local_Is_Directory(folder, &foo), 
+        "Local_Is_Directory() returns true for dir");
+    ASSERT_FALSE(batch, Folder_Local_Is_Directory(folder, &bar), 
+        "Local_Is_Directory() returns false for non-existent entry");
+
+    Folder_Delete(folder, &boffo);
+    Folder_Delete(folder, &foo);
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Local_Find_Folder(TestBatch *batch, set_up_t set_up, 
+                       tear_down_t tear_down)
+{
+    Folder    *folder = set_up();
+    Folder    *local;
+    OutStream *outstream;
+
+    Folder_MkDir(folder, &foo);
+    Folder_MkDir(folder, &foo_bar);
+    outstream = Folder_Open_Out(folder, &boffo);
+    DECREF(outstream);
+    outstream = Folder_Open_Out(folder, &foo_boffo);
+    DECREF(outstream);
+
+    local = Folder_Local_Find_Folder(folder, &nope);
+    ASSERT_TRUE(batch, local == NULL, "Non-existent entry yields NULL");
+
+    local = Folder_Local_Find_Folder(folder, (CharBuf*)&EMPTY);
+    ASSERT_TRUE(batch, local == NULL, "Empty string yields NULL");
+
+    local = Folder_Local_Find_Folder(folder, &foo_bar);
+    ASSERT_TRUE(batch, local == NULL, "nested folder yields NULL");
+
+    local = Folder_Local_Find_Folder(folder, &foo_boffo);
+    ASSERT_TRUE(batch, local == NULL, "nested file yields NULL");
+
+    local = Folder_Local_Find_Folder(folder, &boffo);
+    ASSERT_TRUE(batch, local == NULL, "local file yields NULL");
+    
+    local = Folder_Local_Find_Folder(folder, &bar);
+    ASSERT_TRUE(batch, local == NULL, "name of nested folder yields NULL");
+
+    local = Folder_Local_Find_Folder(folder, &foo);
+    ASSERT_TRUE(batch, 
+        local 
+        && Obj_Is_A(local, FOLDER)
+        && CB_Ends_With(Folder_Get_Path(local), &foo), 
+        "Find local directory");
+
+    Folder_Delete(folder, &foo_bar);
+    Folder_Delete(folder, &foo_boffo);
+    Folder_Delete(folder, &foo);
+    Folder_Delete(folder, &boffo);
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Local_MkDir(TestBatch *batch, set_up_t set_up, tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    bool_t result;
+
+    result = Folder_Local_MkDir(folder, &foo);
+    ASSERT_TRUE(batch, result, "Local_MkDir succeeds and returns true");
+
+    Err_set_error(NULL);
+    result = Folder_Local_MkDir(folder, &foo);
+    ASSERT_FALSE(batch, result, 
+        "Local_MkDir returns false when a dir already exists");
+    ASSERT_TRUE(batch, Err_get_error() != NULL, 
+        "Local_MkDir sets Err_error when a dir already exists");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo), 
+        "Existing dir untouched after failed Local_MkDir");
+
+    {
+        OutStream *outstream = Folder_Open_Out(folder, &boffo);
+        DECREF(outstream);
+        Err_set_error(NULL);
+        result = Folder_Local_MkDir(folder, &foo);
+        ASSERT_FALSE(batch, result, 
+            "Local_MkDir returns false when a file already exists");
+        ASSERT_TRUE(batch, Err_get_error() != NULL, 
+            "Local_MkDir sets Err_error when a file already exists");
+        ASSERT_TRUE(batch, Folder_Exists(folder, &boffo) &&
+            !Folder_Local_Is_Directory(folder, &boffo), 
+            "Existing file untouched after failed Local_MkDir");
+    }
+    
+    Folder_Delete(folder, &foo);
+    Folder_Delete(folder, &boffo);
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Local_Open_Dir(TestBatch *batch, set_up_t set_up, tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    DirHandle *dh = Folder_Local_Open_Dir(folder);
+    ASSERT_TRUE(batch, dh && DH_Is_A(dh, DIRHANDLE), 
+        "Local_Open_Dir returns an DirHandle");
+    DECREF(dh);
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Local_Open_FileHandle(TestBatch *batch, set_up_t set_up, 
+                           tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    FileHandle *fh;
+
+    fh = Folder_Local_Open_FileHandle(folder, &boffo, 
+        FH_CREATE | FH_WRITE_ONLY | FH_EXCLUSIVE);
+    ASSERT_TRUE(batch, fh && Obj_Is_A(fh, FILEHANDLE), 
+        "opened FileHandle");
+    DECREF(fh);
+
+    fh = Folder_Local_Open_FileHandle(folder, &boffo, 
+        FH_CREATE | FH_WRITE_ONLY);
+    ASSERT_TRUE(batch, fh && Obj_Is_A(fh, FILEHANDLE), 
+        "opened FileHandle for append");
+    DECREF(fh);
+
+    Err_set_error(NULL);
+    fh = Folder_Local_Open_FileHandle(folder, &boffo, 
+        FH_CREATE | FH_WRITE_ONLY | FH_EXCLUSIVE);
+    ASSERT_TRUE(batch, fh == NULL, "FH_EXLUSIVE flag prevents clobber");
+    ASSERT_TRUE(batch, Err_get_error() != NULL,
+        "failure due to FH_EXLUSIVE flag sets Err_error");
+
+    fh = Folder_Local_Open_FileHandle(folder, &boffo, FH_READ_ONLY);
+    ASSERT_TRUE(batch, fh && Obj_Is_A(fh, FILEHANDLE), 
+        "opened FileHandle for reading");
+    DECREF(fh);
+
+    Err_set_error(NULL);
+    fh = Folder_Local_Open_FileHandle(folder, &nope, FH_READ_ONLY);
+    ASSERT_TRUE(batch, fh == NULL, 
+        "Can't open non-existent file for reading");
+    ASSERT_TRUE(batch, Err_get_error() != NULL,
+        "Opening non-existent file for reading sets Err_error");
+
+    Folder_Delete(folder, &boffo);
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Local_Delete(TestBatch *batch, set_up_t set_up, tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    OutStream *outstream;
+    
+    outstream = Folder_Open_Out(folder, &boffo);
+    DECREF(outstream);
+    ASSERT_TRUE(batch, Folder_Local_Delete(folder, &boffo), 
+        "Local_Delete on file succeeds");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &boffo), 
+        "File is really gone");
+
+    Folder_Local_MkDir(folder, &foo);
+    outstream = Folder_Open_Out(folder, &foo_boffo);
+    DECREF(outstream);
+
+    Err_set_error(NULL);
+    ASSERT_FALSE(batch, Folder_Local_Delete(folder, &foo), 
+        "Local_Delete on non-empty dir fails");
+
+    Folder_Delete(folder, &foo_boffo);
+    ASSERT_TRUE(batch, Folder_Local_Delete(folder, &foo), 
+        "Local_Delete on empty dir succeeds");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &foo), 
+        "Dir is really gone");
+
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Rename(TestBatch *batch, set_up_t set_up, tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    OutStream *outstream;
+    bool_t result;
+
+    Folder_MkDir(folder, &foo);
+    Folder_MkDir(folder, &foo_bar);
+    outstream = Folder_Open_Out(folder, &boffo);
+    OutStream_Close(outstream);
+    DECREF(outstream);
+
+    /* Move files. */
+
+    result = Folder_Rename(folder, &boffo, &banana); 
+    ASSERT_TRUE(batch, result, "Rename succeeds and returns true");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &banana), 
+        "File exists at new path");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &boffo), 
+        "File no longer exists at old path");
+
+    result = Folder_Rename(folder, &banana, &foo_bar_boffo); 
+    ASSERT_TRUE(batch, result, "Rename to file in nested dir");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_bar_boffo), 
+        "File exists at new path");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &banana), 
+        "File no longer exists at old path");
+
+    result = Folder_Rename(folder, &foo_bar_boffo, &boffo); 
+    ASSERT_TRUE(batch, result, "Rename from file in nested dir");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &boffo), 
+        "File exists at new path");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &foo_bar_boffo), 
+        "File no longer exists at old path");
+
+    outstream = Folder_Open_Out(folder, &foo_boffo);
+    OutStream_Close(outstream);
+    DECREF(outstream);
+    result = Folder_Rename(folder, &boffo, &foo_boffo); 
+    if (result) {
+        PASS(batch, "Rename clobbers on this system");
+        ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+            "File exists at new path");
+        ASSERT_FALSE(batch, Folder_Exists(folder, &boffo), 
+            "File no longer exists at old path");
+    }
+    else {
+        PASS(batch, "Rename does not clobber on this system");
+        ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+            "File exists at new path");
+        ASSERT_TRUE(batch, Folder_Exists(folder, &boffo), 
+            "File still exists at old path");
+        Folder_Delete(folder, &boffo);
+    }
+
+    /* Move Dirs. */
+
+    Folder_MkDir(folder, &baz);
+    result = Folder_Rename(folder, &baz, &boffo); 
+    ASSERT_TRUE(batch, result, "Rename dir");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &boffo), 
+        "Folder exists at new path");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &baz), 
+        "Folder no longer exists at old path");
+
+    result = Folder_Rename(folder, &boffo, &foo_foo); 
+    ASSERT_TRUE(batch, result, "Rename dir into nested subdir");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_foo), 
+        "Folder exists at new path");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &boffo), 
+        "Folder no longer exists at old path");
+
+    result = Folder_Rename(folder, &foo_foo, &foo_bar_baz); 
+    ASSERT_TRUE(batch, result, "Rename dir from nested subdir");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_bar_baz), 
+        "Folder exists at new path");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &foo_foo), 
+        "Folder no longer exists at old path");
+    
+    /* Test failed clobbers. */
+
+    Err_set_error(NULL);
+    result = Folder_Rename(folder, &foo_boffo, &foo_bar); 
+    ASSERT_FALSE(batch, result, "Rename file clobbering dir fails");
+    ASSERT_TRUE(batch, Err_get_error() != NULL, 
+        "Failed rename sets Err_error");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "File still exists at old path");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_bar), 
+        "Dir still exists after failed clobber");
+
+    Err_set_error(NULL);
+    result = Folder_Rename(folder, &foo_bar, &foo_boffo); 
+    ASSERT_FALSE(batch, result, "Rename dir clobbering file fails");
+    ASSERT_TRUE(batch, Err_get_error() != NULL, 
+        "Failed rename sets Err_error");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_bar), 
+        "Dir still exists at old path");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "File still exists after failed clobber");
+
+    /* Test that "renaming" succeeds where to and from are the same. */
+
+    result = Folder_Rename(folder, &foo_boffo, &foo_boffo); 
+    ASSERT_TRUE(batch, result, "Renaming file to itself succeeds");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "File still exists");
+
+    result = Folder_Rename(folder, &foo_bar, &foo_bar); 
+    ASSERT_TRUE(batch, result, "Renaming dir to itself succeeds");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_bar), 
+        "Dir still exists");
+
+    /* Invalid filepaths. */
+
+    Err_set_error(NULL);
+    result = Folder_Rename(folder, &foo_boffo, &nope_nyet); 
+    ASSERT_FALSE(batch, result, "Rename into non-existent subdir fails");
+    ASSERT_TRUE(batch, Err_get_error() != NULL, 
+        "Renaming into non-existent subdir sets Err_error");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "Entry still exists at old path");
+
+    Err_set_error(NULL);
+    result = Folder_Rename(folder, &nope_nyet, &boffo); 
+    ASSERT_FALSE(batch, result, "Rename non-existent file fails");
+    ASSERT_TRUE(batch, Err_get_error() != NULL, 
+        "Renaming non-existent source file sets Err_error");
+
+    Folder_Delete(folder, &foo_bar_baz);
+    Folder_Delete(folder, &foo_bar);
+    Folder_Delete(folder, &foo_boffo);
+    Folder_Delete(folder, &foo);
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Hard_Link(TestBatch *batch, set_up_t set_up, tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    OutStream *outstream;
+    bool_t result;
+
+    Folder_MkDir(folder, &foo);
+    Folder_MkDir(folder, &foo_bar);
+    outstream = Folder_Open_Out(folder, &boffo);
+    DECREF(outstream);
+
+    /* Link files. */
+
+    result = Folder_Hard_Link(folder, &boffo, &banana); 
+    ASSERT_TRUE(batch, result, "Hard_Link succeeds and returns true");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &banana), 
+        "File exists at new path");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &boffo), 
+        "File still exists at old path");
+    Folder_Delete(folder, &boffo);
+
+    result = Folder_Hard_Link(folder, &banana, &foo_bar_boffo); 
+    ASSERT_TRUE(batch, result, "Hard_Link to target within nested dir");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_bar_boffo), 
+        "File exists at new path");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &banana), 
+        "File still exists at old path");
+    Folder_Delete(folder, &banana);
+
+    result = Folder_Hard_Link(folder, &foo_bar_boffo, &foo_boffo); 
+    ASSERT_TRUE(batch, result, "Hard_Link from file in nested dir");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "File exists at new path");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_bar_boffo), 
+        "File still exists at old path");
+    Folder_Delete(folder, &foo_bar_boffo);
+
+    /* Invalid clobbers. */
+
+    outstream = Folder_Open_Out(folder, &boffo);
+    DECREF(outstream);
+    result = Folder_Hard_Link(folder, &foo_boffo, &boffo); 
+    ASSERT_FALSE(batch, result, "Clobber of file fails");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &boffo), 
+        "File still exists at new path");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "File still exists at old path");
+    Folder_Delete(folder, &boffo);
+
+    Folder_MkDir(folder, &baz);
+    result = Folder_Hard_Link(folder, &foo_boffo, &baz); 
+    ASSERT_FALSE(batch, result, "Clobber of dir fails");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &baz), 
+        "Dir still exists at new path");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "File still exists at old path");
+    Folder_Delete(folder, &baz);
+
+    /* Invalid Hard_Link of dir. */
+
+    Folder_MkDir(folder, &baz);
+    result = Folder_Hard_Link(folder, &baz, &banana); 
+    ASSERT_FALSE(batch, result, "Hard_Link dir fails");
+    ASSERT_FALSE(batch, Folder_Exists(folder, &banana), 
+        "Nothing at new path");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &baz), 
+        "Folder still exists at old path");
+    Folder_Delete(folder, &baz);
+
+    /* Test that linking to yourself fails. */
+
+    result = Folder_Hard_Link(folder, &foo_boffo, &foo_boffo); 
+    ASSERT_FALSE(batch, result, "Hard_Link file to itself fails");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "File still exists");
+
+    /* Invalid filepaths. */
+
+    Err_set_error(NULL);
+    result = Folder_Rename(folder, &foo_boffo, &nope_nyet); 
+    ASSERT_FALSE(batch, result, "Hard_Link into non-existent subdir fails");
+    ASSERT_TRUE(batch, Err_get_error() != NULL, 
+        "Hard_Link into non-existent subdir sets Err_error");
+    ASSERT_TRUE(batch, Folder_Exists(folder, &foo_boffo), 
+        "Entry still exists at old path");
+
+    Err_set_error(NULL);
+    result = Folder_Rename(folder, &nope_nyet, &boffo); 
+    ASSERT_FALSE(batch, result, "Hard_Link non-existent source file fails");
+    ASSERT_TRUE(batch, Err_get_error() != NULL, 
+        "Hard_Link non-existent source file sets Err_error");
+
+    Folder_Delete(folder, &foo_bar);
+    Folder_Delete(folder, &foo_boffo);
+    Folder_Delete(folder, &foo);
+    DECREF(folder);
+    tear_down();
+}
+
+static void
+test_Close(TestBatch *batch, set_up_t set_up, tear_down_t tear_down)
+{
+    Folder *folder = set_up();
+    Folder_Close(folder);
+    PASS(batch, "Close() concludes without incident");
+    Folder_Close(folder);
+    Folder_Close(folder);
+    PASS(batch, "Calling Close() multiple times is safe");
+    DECREF(folder);
+    tear_down();
+}
+
+u32_t
+TestFolderCommon_num_tests()
+{
+    return 99;
+}
+
+void
+TestFolderCommon_run_tests(void *test_batch, set_up_t set_up, 
+                           tear_down_t tear_down)
+{
+    TestBatch *batch = (TestBatch*)test_batch;
+
+    test_Local_Exists(batch, set_up, tear_down);
+    test_Local_Is_Directory(batch, set_up, tear_down);
+    test_Local_Find_Folder(batch, set_up, tear_down);
+    test_Local_MkDir(batch, set_up, tear_down);
+    test_Local_Open_Dir(batch, set_up, tear_down);
+    test_Local_Open_FileHandle(batch, set_up, tear_down);
+    test_Local_Delete(batch, set_up, tear_down);
+    test_Rename(batch, set_up, tear_down);
+    test_Hard_Link(batch, set_up, tear_down);
+    test_Close(batch, set_up, tear_down);
+}
+
+/* Copyright 2009 The 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.
+ */
+

Propchange: lucene/lucy/trunk/core/Lucy/Test/Store/TestFolderCommon.c
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/lucy/trunk/perl/lib/Lucy.pm
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/perl/lib/Lucy.pm?rev=888114&r1=888113&r2=888114&view=diff
==============================================================================
--- lucene/lucy/trunk/perl/lib/Lucy.pm (original)
+++ lucene/lucy/trunk/perl/lib/Lucy.pm Mon Dec  7 20:12:30 2009
@@ -165,6 +165,12 @@
 }
 
 {
+    package Lucy::Store::FSFolder;
+    use File::Spec::Functions qw( rel2abs );
+    sub absolutify { return rel2abs( $_[1] ) } 
+}
+
+{
     package Lucy::Util::Json;
     use Lucy::Util::ToolSet qw( to_lucy );
 

Added: lucene/lucy/trunk/perl/lib/Lucy/Store/FSFolder.pm
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/perl/lib/Lucy/Store/FSFolder.pm?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/perl/lib/Lucy/Store/FSFolder.pm (added)
+++ lucene/lucy/trunk/perl/lib/Lucy/Store/FSFolder.pm Mon Dec  7 20:12:30 2009
@@ -0,0 +1,32 @@
+use Lucy;
+
+1;
+
+__END__
+
+__BINDING__
+
+Boilerplater::Binding::Perl::Class->register(
+    parcel            => "Lucy",
+    class_name        => "Lucy::Store::FSFolder",
+    bind_constructors => ["new"],
+);
+
+__COPYRIGHT__
+
+    /** 
+     * Copyright 2009 The 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.
+     */  
+

Propchange: lucene/lucy/trunk/perl/lib/Lucy/Store/FSFolder.pm
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/lucy/trunk/perl/lib/Lucy/Test.pm
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/perl/lib/Lucy/Test.pm?rev=888114&r1=888113&r2=888114&view=diff
==============================================================================
--- lucene/lucy/trunk/perl/lib/Lucy/Test.pm (original)
+++ lucene/lucy/trunk/perl/lib/Lucy/Test.pm Mon Dec  7 20:12:30 2009
@@ -46,15 +46,18 @@
     else if (strEQ(package, "TestCompoundFileWriter")) {
         lucy_TestCFWriter_run_tests();
     }
-    else if (strEQ(package, "TestFSFileHandle")) {
-        lucy_TestFSFH_run_tests();
-    }
     else if (strEQ(package, "TestFileHandle")) {
         lucy_TestFH_run_tests();
     }
     else if (strEQ(package, "TestFolder")) {
         lucy_TestFolder_run_tests();
     }
+    else if (strEQ(package, "TestFSFileHandle")) {
+        lucy_TestFSFH_run_tests();
+    }
+    else if (strEQ(package, "TestFSFolder")) {
+        lucy_TestFSFolder_run_tests();
+    }
     else if (strEQ(package, "TestInStream")) {
         lucy_TestInStream_run_tests();
     }

Added: lucene/lucy/trunk/perl/t/core/103-fs_folder.t
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/perl/t/core/103-fs_folder.t?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/perl/t/core/103-fs_folder.t (added)
+++ lucene/lucy/trunk/perl/t/core/103-fs_folder.t Mon Dec  7 20:12:30 2009
@@ -0,0 +1,6 @@
+use strict;
+use warnings;
+
+use Lucy::Test;
+Lucy::Test::run_tests("TestFSFolder");
+

Propchange: lucene/lucy/trunk/perl/t/core/103-fs_folder.t
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/lucy/trunk/perl/xs/Lucy/Store/FSFolder.c
URL: http://svn.apache.org/viewvc/lucene/lucy/trunk/perl/xs/Lucy/Store/FSFolder.c?rev=888114&view=auto
==============================================================================
--- lucene/lucy/trunk/perl/xs/Lucy/Store/FSFolder.c (added)
+++ lucene/lucy/trunk/perl/xs/Lucy/Store/FSFolder.c Mon Dec  7 20:12:30 2009
@@ -0,0 +1,12 @@
+#include "Lucy/Util/ToolSet.h"
+#include "Lucy/Object/Host.h"
+#include "Lucy/Store/FSFolder.h"
+
+CharBuf*
+FSFolder_absolutify(const CharBuf *path)
+{
+   
+    return Host_callback_str(FSFOLDER, "absolutify", 1, 
+        ARG_STR("path", path));
+}
+

Propchange: lucene/lucy/trunk/perl/xs/Lucy/Store/FSFolder.c
------------------------------------------------------------------------------
    svn:eol-style = native