You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by rh...@apache.org on 2014/05/02 15:55:51 UTC

svn commit: r1591910 - in /db/derby/code/trunk/java: engine/org/apache/derby/database/ engine/org/apache/derby/iapi/util/ engine/org/apache/derby/impl/db/ engine/org/apache/derby/impl/io/vfmem/ engine/org/apache/derby/impl/store/raw/ engine/org/apache/...

Author: rhillegas
Date: Fri May  2 13:55:51 2014
New Revision: 1591910

URL: http://svn.apache.org/r1591910
Log:
DERBY-590: Add backup/restore support for Lucene indexes and prevent the Lucene plugin from being used in an encrypted database; commit derby-590-26-ad-backupRestoreEncryption.diff.

Added:
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneBackupTest.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/util/StringUtil.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/io/vfmem/DataStore.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml
    db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/DerbyLuceneDir.java
    db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneListIndexesVTI.java
    db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneSupport.java
    db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSuite.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSupportPermsTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/luceneSupport.policy

Modified: db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java Fri May  2 13:55:51 2014
@@ -47,6 +47,8 @@ import java.io.File;
  */
 public interface Database
 {
+    // where the Lucene plugin writes its indexes
+    public  static  final   String  LUCENE_DIR = "LUCENE";
 
 	/**
 	 * Tells whether the Database is configured as read-only, or the

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/util/StringUtil.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/util/StringUtil.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/util/StringUtil.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/util/StringUtil.java Fri May  2 13:55:51 2014
@@ -585,5 +585,19 @@ public class StringUtil 
 		}
 		return result.toString();
 	}
+
+    /**
+     * Get the short database name from the canonical name.
+     */
+    public  static  String  shortDBName
+        (
+         String canonicalDBName,
+         char   separatorChar
+         )
+    {
+        int         lastSep = canonicalDBName.lastIndexOf( separatorChar );
+        return canonicalDBName.substring( lastSep + 1 );
+    }
+    
 }
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java Fri May  2 13:55:51 2014
@@ -43,9 +43,10 @@ import org.apache.derby.iapi.db.Database
 import org.apache.derby.iapi.db.DatabaseContext;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.sql.execute.ExecutionFactory;
+import org.apache.derby.iapi.store.raw.data.DataFactory;
 import org.apache.derby.iapi.types.DataValueFactory;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
-
+import org.apache.derby.iapi.util.StringUtil;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionFactory;
 
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
@@ -62,12 +63,19 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.jdbc.AuthenticationService;
 import org.apache.derby.iapi.services.uuid.UUIDFactory;
 import org.apache.derby.impl.sql.execute.JarUtil;
+import org.apache.derby.iapi.services.io.FileUtil;
 import org.apache.derby.io.StorageFile;
+import org.apache.derby.io.StorageFactory;
 import org.apache.derby.catalog.UUID;
 
 import org.apache.derby.iapi.store.replication.slave.SlaveFactory;
 
+import java.io.File;
+import java.io.IOException;
 import java.io.Serializable;
+import java.security.AccessController;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
 import java.sql.SQLException;
 import java.util.Properties;
 import java.util.Dictionary;
@@ -428,6 +436,10 @@ public class BasicDatabase implements Mo
     {
 		try {
 			af.backup(backupDir, wait);
+            if ( luceneLoaded() )
+            {
+                backupLucene( backupDir );
+            }
 		} catch (StandardException se) {
 			throw PublicAPI.wrapStandardException(se);
 		}
@@ -443,6 +455,10 @@ public class BasicDatabase implements Mo
 			af.backupAndEnableLogArchiveMode(backupDir, 
                                              deleteOnlineArchivedLogFiles,
                                              wait); 
+            if ( luceneLoaded() )
+            {
+                backupLucene( backupDir );
+            }
 		} catch (StandardException se) {
 			throw PublicAPI.wrapStandardException(se);
 		}
@@ -827,4 +843,102 @@ public class BasicDatabase implements Mo
 		return fr.getAsFile(externalName, generationId);
 	}
 
+    ////////////////////////////////////////////////////////////////////////
+    //
+    // SUPPORT FOR BACKING UP LUCENE DIRECTORY
+    //
+    ////////////////////////////////////////////////////////////////////////
+
+    /**
+     * <p>
+     * Return true if the Lucene plugin is loaded.
+     * </p>
+     */
+    private boolean luceneLoaded()
+        throws StandardException
+    {
+        try {
+            return AccessController.doPrivileged
+                (
+                 new PrivilegedExceptionAction<Boolean>()
+                 {
+                     public Boolean run()
+                         throws StandardException
+                     {
+                         return getLuceneDir().exists();
+                     }
+                 }
+                 ).booleanValue();
+        }
+        catch (PrivilegedActionException pae) { throw StandardException.plainWrapException( pae ); }
+    }
+
+    /** Get the location of the Lucene indexes */
+    private StorageFile getLuceneDir()
+        throws StandardException
+    {
+        StorageFactory  dir = getStorageFactory();
+		
+		return dir.newStorageFile( Database.LUCENE_DIR );
+    }
+
+    /**
+     * <p>
+     * Get the database StorageFactory.
+     * </p>
+     */
+    private StorageFactory  getStorageFactory()
+        throws StandardException
+    {
+        DataFactory dataFactory = (DataFactory) Monitor.findServiceModule( this, DataFactory.MODULE );
+
+        return dataFactory.getStorageFactory();
+    }
+
+    /**
+     * <p>
+     * Backup Lucene indexes to the backup directory. This assumes
+     * that the rest of the database has been backup up and sanity
+     * checks have been run.
+     * </p>
+     */
+    private void    backupLucene( String backupDir )
+        throws StandardException
+    {
+        try {
+            File            backupRoot = new File( backupDir );
+            StorageFactory  storageFactory = getStorageFactory();
+            String      canonicalDbName = storageFactory.getCanonicalName();
+            String      dbname = StringUtil.shortDBName( canonicalDbName, storageFactory.getSeparator() );
+            File        backupDB = new File( backupRoot, dbname );
+        
+            final   File            targetDir = new File( backupDB, Database.LUCENE_DIR );
+            final   StorageFile sourceDir = getLuceneDir();
+
+            AccessController.doPrivileged
+                (
+                 new PrivilegedExceptionAction<Object>()
+                 {
+                     public Boolean run()
+                         throws StandardException
+                     {
+                         if ( !FileUtil.copyDirectory( getStorageFactory(), sourceDir, targetDir, null, null, true ) )
+                         {
+                             throw StandardException.newException
+                                 (
+                                  SQLState.UNABLE_TO_COPY_FILE_FROM_BACKUP,
+                                  sourceDir.getPath(),
+                                  targetDir.getAbsolutePath()
+                                  );
+                         }
+                         
+                         return null;
+                     }
+                 }
+                 );
+        }
+        catch (IOException ioe) { throw StandardException.plainWrapException( ioe ); }
+        catch (PrivilegedActionException pae) { throw StandardException.plainWrapException( pae ); }
+    }
+
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/io/vfmem/DataStore.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/io/vfmem/DataStore.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/io/vfmem/DataStore.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/io/vfmem/DataStore.java Fri May  2 13:55:51 2014
@@ -262,7 +262,12 @@ public final class DataStore {
             while (paths.hasNext()) {
                 candidate = paths.next();
                 if (candidate.startsWith(nPath)) {
-                    children.add(candidate.substring(nPath.length()));
+                    candidate = candidate.substring(nPath.length());
+                    // don't include grandchildren
+                    if ( candidate.indexOf( PathUtil.SEP_STR ) < 0 )
+                    {
+                        children.add(candidate);
+                    }
                 }
             }
         }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java Fri May  2 13:55:51 2014
@@ -51,6 +51,7 @@ import org.apache.derby.iapi.store.raw.x
 import org.apache.derby.iapi.store.raw.data.DataFactory;
 import org.apache.derby.iapi.store.raw.log.LogFactory;
 import org.apache.derby.iapi.store.raw.log.LogInstant;
+import org.apache.derby.iapi.util.StringUtil;
 import org.apache.derby.impl.services.monitor.UpdateServiceProperties;
 
 import org.apache.derby.io.StorageFactory;
@@ -718,10 +719,7 @@ public final class RawStore implements R
 			// get name of the current db, ie. database directory of current db.
 			StorageFile dbase           = storageFactory.newStorageFile(null); 
             String      canonicalDbName = storageFactory.getCanonicalName();
-            int         lastSep         = 
-                canonicalDbName.lastIndexOf(storageFactory.getSeparator());
-			String      dbname          = 
-                canonicalDbName.substring(lastSep + 1);
+            String      dbname = StringUtil.shortDBName( canonicalDbName, storageFactory.getSeparator() );
 
 			// append to end of history file
 			historyFile = 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java Fri May  2 13:55:51 2014
@@ -26,6 +26,7 @@ import org.apache.derby.iapi.reference.M
 
 import org.apache.derby.iapi.services.info.ProductVersionHolder;
 
+import org.apache.derby.database.Database;
 import org.apache.derby.iapi.services.cache.CacheFactory;
 import org.apache.derby.iapi.services.cache.CacheManager;
 import org.apache.derby.iapi.services.cache.Cacheable;
@@ -299,6 +300,16 @@ public class BaseDataFileFactory
             }
         }
 
+        // you can't encrypt a database if the Lucene plugin is loaded
+        if ( luceneLoaded() )
+        {
+            String  encryptionProp = startParams.getProperty( Attribute.DATA_ENCRYPTION );
+            if ( (encryptionProp != null) && "TRUE".equals( encryptionProp.toUpperCase() ) )
+            {
+                throw StandardException.newException( SQLState.LUCENE_ENCRYPTED_DB );
+            }
+        }
+
         if (storageFactory instanceof WritableStorageFactory)
             writableStorageFactory = (WritableStorageFactory) storageFactory;
 
@@ -2637,7 +2648,7 @@ public class BaseDataFileFactory
             for (int i = 0; i < cfilelist.length; i++) 
             {
                 //delete only the seg* directories in the database home
-                if(cfilelist[i].startsWith("seg"))
+                if(cfilelist[i].startsWith("seg") || Database.LUCENE_DIR.equals( cfilelist[i] ))
                 {
                     csegdir = storageFactory.newStorageFile( cfilelist[i]);
                     if(!csegdir.deleteAll())
@@ -2655,7 +2666,7 @@ public class BaseDataFileFactory
         for (int i = 0; i < bfilelist.length; i++) 
         {
             //copy only the seg* directories and copy them from backup
-            if (bfilelist[i].startsWith("seg"))
+            if (bfilelist[i].startsWith("seg") || Database.LUCENE_DIR.equals( bfilelist[i] ))
             {
                 csegdir = storageFactory.newStorageFile( bfilelist[i]);
                 File bsegdir1 = new java.io.File(backupRoot, bfilelist[i]);
@@ -2702,6 +2713,28 @@ public class BaseDataFileFactory
 		return readOnly;
 	}
 
+    /** Return true if the Lucene plugin is loaded */
+    public  boolean luceneLoaded()
+        throws StandardException
+    {
+        try {
+            return AccessController.doPrivileged
+                (
+                 new PrivilegedExceptionAction<Boolean>()
+                 {
+                     public Boolean run()
+                     {
+                         StorageFactory storageFactory = getStorageFactory();
+                         StorageFile luceneDir = storageFactory.newStorageFile( Database.LUCENE_DIR );
+
+                         return luceneDir.exists();
+                     }
+                 }
+                 ).booleanValue();
+        }
+        catch (PrivilegedActionException pae) { throw StandardException.plainWrapException( pae ); }
+    }
+
     /**
      * @return The StorageFactory used by this dataFactory
      */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml Fri May  2 13:55:51 2014
@@ -2408,6 +2408,11 @@ Guide.
                 <arg>indexVersion</arg>
             </msg>
 
+	        <msg>
+                <name>42XBL</name>
+                <text>Lucene indexes cannot be created in an encrypted database and, conversely, a database containing a Lucene index cannot be encrypted.</text>
+            </msg>
+
             <msg>
                 <name>42Y00</name>
                 <text>Class '{0}' does not implement org.apache.derby.iapi.db.AggregateDefinition and thus cannot be used as an aggregate expression.</text>

Modified: db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/DerbyLuceneDir.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/DerbyLuceneDir.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/DerbyLuceneDir.java (original)
+++ db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/DerbyLuceneDir.java Fri May  2 13:55:51 2014
@@ -36,6 +36,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.SingleInstanceLockFactory;
 
+import org.apache.derby.database.Database;
 import org.apache.derby.io.StorageFactory;
 import org.apache.derby.io.StorageFile;
 import org.apache.derby.shared.common.reference.SQLState;
@@ -62,6 +63,9 @@ class DerbyLuceneDir extends Directory
     // constructor args
     private final   StorageFactory  _storageFactory;
     private final   StorageFile     _directory;
+    private final   String              _schema;
+    private final   String              _table;
+    private final   String              _textcol;
 
     // files open for output which may need to be sync'd
     private HashMap<String,DerbyIndexOutput>    _outputFiles = new HashMap<String,DerbyIndexOutput>();
@@ -82,11 +86,12 @@ class DerbyLuceneDir extends Directory
      * Lookup a directory, creating its path as necessary.
      * </p>
      */
-    static  synchronized    DerbyLuceneDir  getDirectory( StorageFactory storageFactory, String directoryPath )
+    static  synchronized    DerbyLuceneDir  getDirectory
+        ( StorageFactory storageFactory, String schema, String table, String textcol )
         throws SQLException
     {
         try {
-            DerbyLuceneDir  candidate = new DerbyLuceneDir( storageFactory, directoryPath );
+            DerbyLuceneDir  candidate = new DerbyLuceneDir( storageFactory, schema, table, textcol );
             String              key = getKey( candidate );
             DerbyLuceneDir  result = _openDirectories.get( key );
 
@@ -97,8 +102,6 @@ class DerbyLuceneDir extends Directory
                 _openDirectories.put( key, result );
             }
 
-            createPath( result._directory );
-
             return result;
         }
         catch (IOException ioe) { throw LuceneSupport.wrap( ioe ); }
@@ -116,7 +119,10 @@ class DerbyLuceneDir extends Directory
     }
 
     /** Get the key associated with a directory */
-    private static  String  getKey( DerbyLuceneDir dir ) { return dir._directory.getPath(); }
+    private static  String  getKey( DerbyLuceneDir dir )
+    {
+        return dir._directory.getPath();
+    }
     
     /**
      * <p>
@@ -125,10 +131,14 @@ class DerbyLuceneDir extends Directory
      * Creates the directory if it does not already exist.
      * </p>
      */
-    private DerbyLuceneDir( StorageFactory storageFactory, String directoryPath )
+    private DerbyLuceneDir( StorageFactory storageFactory, String schema, String table, String textcol )
+        throws IOException, PrivilegedActionException, SQLException
     {
         _storageFactory = storageFactory;
-        _directory = _storageFactory.newStorageFile( directoryPath );
+        _schema = schema;
+        _table = table;
+        _textcol = textcol;
+        _directory = createPath( _storageFactory, _schema, _table, _textcol );
     }
 
     /////////////////////////////////////////////////////////////////////
@@ -330,21 +340,46 @@ class DerbyLuceneDir extends Directory
 	/**
 	 * Create the path if necessary.
 	 */
-    private static void createPath( final StorageFile directory )
-        throws IOException, PrivilegedActionException
+    private static StorageFile createPath
+        ( final StorageFactory storageFactory, final String schema, final String table, final String textcol )
+        throws SQLException, IOException, PrivilegedActionException
+    {
+        StorageFile    luceneDir = createPathLeg( storageFactory, null, Database.LUCENE_DIR );
+        StorageFile    schemaDir = createPathLeg( storageFactory, luceneDir, schema );
+        StorageFile    tableDir = createPathLeg( storageFactory, schemaDir, table );
+        StorageFile    indexDir = createPathLeg( storageFactory, tableDir, textcol );
+
+        return indexDir;
+    }
+
+	/**
+	 * Create the path if necessary.
+	 */
+    private static StorageFile createPathLeg
+        ( final StorageFactory storageFactory, final StorageFile parentDir, final String fileName )
+        throws SQLException, IOException, PrivilegedActionException
     {
-        AccessController.doPrivileged
+        return AccessController.doPrivileged
             (
-             new PrivilegedExceptionAction<Object>()
+             new PrivilegedExceptionAction<StorageFile>()
              {
-                 public Object run() throws IOException
+                 public StorageFile run() throws IOException, SQLException
                  {
-                     directory.mkdirs();
-                     return null;
+                     String         normalizedName = LuceneSupport.derbyIdentifier( fileName );
+                     StorageFile    file = parentDir == null ?
+                         storageFactory.newStorageFile( normalizedName  ) :
+                         storageFactory.newStorageFile( parentDir, normalizedName );
+
+                     if ( !file.exists() ) { file.mkdir(); }
+                     if ( !file.exists() )
+                     {
+                         throw LuceneSupport.newSQLException( SQLState.SERVICE_DIRECTORY_CREATE_ERROR, normalizedName );
+                     }
+                     else { return file; }
                  }
              }
              );
     }
-
+    
 }
 

Modified: db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneListIndexesVTI.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneListIndexesVTI.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneListIndexesVTI.java (original)
+++ db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneListIndexesVTI.java Fri May  2 13:55:51 2014
@@ -32,6 +32,7 @@ import java.text.DateFormat;
 import java.util.ArrayList;
 import java.util.Properties;
 
+import org.apache.derby.database.Database;
 import org.apache.derby.io.StorageFactory;
 import org.apache.derby.io.StorageFile;
 
@@ -78,7 +79,7 @@ public class LuceneListIndexesVTI extend
         connection = LuceneSupport.getDefaultConnection();
         StorageFactory  dir = LuceneSupport.getStorageFactory( connection );
 		
-		StorageFile luceneDir = dir.newStorageFile( LuceneSupport.LUCENE_DIR );
+		StorageFile luceneDir = dir.newStorageFile( Database.LUCENE_DIR );
         ArrayList<StorageFile> allIndexes = new ArrayList<StorageFile>();
 
         StorageFile[]  schemas = listDirectories( dir, luceneDir );

Modified: db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneSupport.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneSupport.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneSupport.java (original)
+++ db/derby/code/trunk/java/optional/org/apache/derby/optional/lucene/LuceneSupport.java Fri May  2 13:55:51 2014
@@ -46,6 +46,7 @@ import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.Properties;
 
+import org.apache.derby.database.Database;
 import org.apache.derby.iapi.sql.conn.ConnectionUtil;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.sql.dictionary.OptionalTool;
@@ -99,8 +100,6 @@ public class LuceneSupport implements Op
     private static  final   String  UPDATE_INDEX = LUCENE_SCHEMA + "." + "updateIndex";
     private static  final   String  SEPARATOR = "__";
 
-    static  final   String  LUCENE_DIR = "lucene";
-
     // names of columns in all query table functions
     private static  final   String  SCORE = "SCORE";
     private static  final   String  DOCUMENT_ID = "DOCUMENTID";
@@ -159,6 +158,15 @@ public class LuceneSupport implements Op
         Connection  conn = getDefaultConnection();
         mustBeDBO( conn );
 
+        //
+        // Lucene indexes are not allowed in encrypted databases. They leak
+        // encrypted data in plaintext.
+        //
+        if ( getDataFactory( conn ).databaseEncrypted() )
+        {
+            throw newSQLException( SQLState.LUCENE_ENCRYPTED_DB );
+        }
+
         if ( luceneSchemaExists( conn ) )
         {
             throw newSQLException( SQLState.LUCENE_ALREADY_LOADED );
@@ -220,6 +228,8 @@ public class LuceneSupport implements Op
 		executeDDL( conn, updateProcedure.toString() );
 
         if ( sqlAuthorizationEnabled ) { grantPermissions(); }
+
+        createLuceneDir( conn );
 	}
 
     /**
@@ -291,7 +301,7 @@ public class LuceneSupport implements Op
         //
         try {
             StorageFactory  storageFactory = getStorageFactory( conn );
-            StorageFile     luceneDir = storageFactory.newStorageFile( LUCENE_DIR );
+            StorageFile     luceneDir = storageFactory.newStorageFile( Database.LUCENE_DIR );
             if ( exists( luceneDir ) ) { deleteFile( luceneDir ); }
         }
         catch (IOException ioe) { throw wrap( ioe ); }
@@ -1349,7 +1359,7 @@ public class LuceneSupport implements Op
     }
 	
     /** Convert a raw string into a properly cased and escaped Derby identifier */
-    private static  String  derbyIdentifier( String rawString )
+    static  String  derbyIdentifier( String rawString )
         throws SQLException
     {
         try {
@@ -1711,6 +1721,32 @@ public class LuceneSupport implements Op
         return (Analyzer) method.invoke( null );
 	}
 
+	/**
+	 * Add a document to a Lucene index wrier.
+	 */
+    private static void createLuceneDir( final Connection conn )
+        throws SQLException
+    {
+        try {
+            AccessController.doPrivileged
+                (
+                 new PrivilegedExceptionAction<Object>()
+                 {
+                     public Object run() throws SQLException
+                     {
+                         StorageFactory storageFactory = getStorageFactory( conn );
+                         StorageFile    luceneDir = storageFactory.newStorageFile( Database.LUCENE_DIR );
+
+                         luceneDir.mkdir();
+		
+                         return null;
+                     }
+                 }
+                 );
+        }
+        catch (PrivilegedActionException pae) { throw wrap( pae ); }
+    }
+
     /////////////////////////////////////////////////////////////////////
     //
     //  DERBY STORE
@@ -1726,38 +1762,26 @@ public class LuceneSupport implements Op
         throws SQLException
     {
         StorageFactory  storageFactory = getStorageFactory( conn );
-        String  relativePath = getRelativeIndexPath( schema, table, textcol );
-        DerbyLuceneDir  result = DerbyLuceneDir.getDirectory( storageFactory, relativePath );
+        DerbyLuceneDir  result = DerbyLuceneDir.getDirectory( storageFactory, schema, table, textcol );
 
         return result;
     }
     
-    /**
-     * <p>
-     * Get the relative path of the index in the database.
-     * </p>
-     */
-    private static String getRelativeIndexPath( String schema, String table, String textcol )
+    /** Get the StorageFactory of the connected database */
+    static  StorageFactory  getStorageFactory( Connection conn )
         throws SQLException
     {
-        return
-            LUCENE_DIR + File.separator +
-            derbyIdentifier( schema ) + File.separator +
-            derbyIdentifier( table ) + File.separator +
-            derbyIdentifier( textcol ) + File.separator;
+        return getDataFactory( conn ).getStorageFactory();
     }
 
-    
-    /** Get the StorageFactory of the connected database */
-    static  StorageFactory  getStorageFactory( Connection conn )
+    /** Get the DataFactory of the connected database */
+    static  DataFactory  getDataFactory( Connection conn )
         throws SQLException
     {
         try {
             Object monitor = Monitor.findService
                 ( Property.DATABASE_MODULE, ((EmbedConnection) conn).getDBName() ) ;
-            DataFactory dataFactory = (DataFactory) Monitor.findServiceModule( monitor, DataFactory.MODULE );
-
-            return dataFactory.getStorageFactory();
+            return (DataFactory) Monitor.findServiceModule( monitor, DataFactory.MODULE );
         }
         catch (StandardException se) { throw wrap( se ); }
     }

Modified: db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java (original)
+++ db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java Fri May  2 13:55:51 2014
@@ -1188,6 +1188,7 @@ public interface SQLState {
     String LUCENE_BAD_INDEX                                               = "42XBI";
     String LUCENE_BAD_COLUMN_NAME                                   = "42XBJ";
     String LUCENE_BAD_VERSION                                           = "42XBK";
+    String LUCENE_ENCRYPTED_DB                                           = "42XBL";
     
 	// org.apache.derby.impl.sql.execute.rts
 	String RTS_ATTACHED_TO											   = "43X00.U";

Added: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneBackupTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneBackupTest.java?rev=1591910&view=auto
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneBackupTest.java (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneBackupTest.java Fri May  2 13:55:51 2014
@@ -0,0 +1,236 @@
+/*
+
+   Derby - Class org.apache.derbyTesting.functionTests.tests.lang.LuceneBackupTest
+
+   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.
+
+ */
+
+package org.apache.derbyTesting.functionTests.tests.lang;
+
+import java.io.File;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+
+import junit.framework.Test;
+import junit.framework.TestSuite;
+
+import org.apache.derbyTesting.junit.BaseJDBCTestCase;
+import org.apache.derbyTesting.junit.JDBC;
+import org.apache.derbyTesting.junit.DatabasePropertyTestSetup;
+import org.apache.derbyTesting.junit.LocaleTestSetup;
+import org.apache.derbyTesting.junit.SecurityManagerSetup;
+import org.apache.derbyTesting.junit.TestConfiguration;
+import org.apache.derbyTesting.junit.CleanDatabaseTestSetup;
+import org.apache.derbyTesting.junit.SupportFilesSetup;
+
+/**
+ * <p>
+ * Test backup and restore of databases with Lucene indexes.
+ * </p>
+ */
+public class LuceneBackupTest extends GeneratedColumnsHelper
+{
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTANTS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    private static  final   String      DB_NAME = "lucenebackupdb";
+    private static  final   String      BACKUP_DIR = "extinout/backupdir";
+    private static  final   String      BACKUP_DIR2 = "extinout/backupdir2";
+
+    private static  final   String      TEST_DBO = "TEST_DBO";
+    private static  final   String      RUTH = "RUTH";
+    private static  final   String      ALICE = "ALICE";
+    private static  final   String      FRANK = "FRANK";
+    private static  final   String[]    LEGAL_USERS = { TEST_DBO, ALICE, RUTH, FRANK  };
+
+    private static  final   String      POLICY_FILE = "org/apache/derbyTesting/functionTests/tests/lang/luceneSupport.policy";
+
+    private static  final   String      ENGLISH_ANALYZER =
+        "org.apache.derbyTesting.functionTests.tests.lang.LuceneCoarseAuthorizationTest.getEnglishAnalyzer";
+
+    private static  final   String      LOAD_TOOL = "call syscs_util.syscs_register_tool( 'luceneSupport', true )";
+    private static  final   String      UNLOAD_TOOL = "call syscs_util.syscs_register_tool( 'luceneSupport', false )";
+    private static  final   String      INDEX_POEMS =
+        "call LuceneSupport.createIndex( 'ruth', 'poems', 'poemText', '" + ENGLISH_ANALYZER + "' )";
+    private static  final   String      DROP_POEMS_INDEX = "call LuceneSupport.dropIndex( 'ruth', 'poems', 'poemText' )";
+
+    private static  final   String      READ_POEMS_INDEX =
+        "select p.originalAuthor, i.score\n" +
+        "from ruth.poems p, table ( ruth.poems__poemText( 'star', null, 1000, 0 ) ) i\n" +
+        "where p.poemID = i.poemID and p.versionStamp = i.versionStamp\n" +
+        "order by i.score desc\n";
+    private static  final   String[][]  DEFAULT_POEMS_RESULT =
+        new String[][]
+        {
+            { "Walt Whitman", "0.26756266" },
+            { "Lord Byron", "0.22933942" },
+            { "John Milton", "0.22933942" },
+        };
+
+    private static  final   String  GOOD_SHUTDOWN = "08006";
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // STATE
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // CONSTRUCTOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /**
+     * Create a new instance.
+     */
+
+    public LuceneBackupTest( String name )
+    {
+        super( name );
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // JUnit BEHAVIOR
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+
+    /**
+     * Construct top level suite in this JUnit test
+     */
+    public static Test suite()
+    {
+        TestSuite suite = (TestSuite) TestConfiguration.embeddedSuite( LuceneBackupTest.class );
+
+        Test        secureTest = new SecurityManagerSetup( suite, POLICY_FILE );
+        Test        authenticatedTest = DatabasePropertyTestSetup.builtinAuthentication
+            ( secureTest, LEGAL_USERS, "LuceneBackupPermissions" );
+        Test        authorizedTest = TestConfiguration.sqlAuthorizationDecoratorSingleUse( authenticatedTest, DB_NAME, true );
+        Test        supportFilesTest = new SupportFilesSetup( authorizedTest );
+
+        return supportFilesTest;
+    }
+
+    ///////////////////////////////////////////////////////////////////////////////////
+    //
+    // TESTS
+    //
+    ///////////////////////////////////////////////////////////////////////////////////
+
+    /**
+     * <p>
+     * Test basic functionality of backup/restore.
+     * </p>
+     */
+    public  void    test_001_basic()
+        throws Exception
+    {
+        Connection  dboConnection = openUserConnection( TEST_DBO );
+        Connection  ruthConnection = openUserConnection( RUTH );
+
+        LuceneCoarseAuthorizationTest.createSchema( ruthConnection );
+
+        goodStatement( dboConnection, LOAD_TOOL );
+        goodStatement( ruthConnection, INDEX_POEMS );
+
+        // verify that everything looks good
+        assertResults
+            (
+             ruthConnection,
+             READ_POEMS_INDEX,
+             DEFAULT_POEMS_RESULT,
+             false
+             );
+
+        // now backup the database from disk to disk
+        goodStatement( dboConnection, "call syscs_util.syscs_backup_database( '" + BACKUP_DIR + "' )" );
+
+        goodStatement( ruthConnection, DROP_POEMS_INDEX );
+        goodStatement( dboConnection, UNLOAD_TOOL );
+
+        LuceneCoarseAuthorizationTest.dropSchema( ruthConnection );
+
+        //
+        // Now restore the database and verify it.
+        //
+        String  source = BACKUP_DIR + File.separator + DB_NAME;
+        String  dboPassword = getTestConfiguration().getPassword( TEST_DBO );
+        String  ruthPassword = getTestConfiguration().getPassword( RUTH );
+        dboConnection = DriverManager.getConnection
+            ( "jdbc:derby:memory:lbt1;user=" + TEST_DBO + ";password=" + dboPassword + ";restoreFrom=" + source );
+        ruthConnection = DriverManager.getConnection
+            ( "jdbc:derby:memory:lbt1;user=" + RUTH + ";password=" + ruthPassword );
+        assertResults
+            (
+             ruthConnection,
+             READ_POEMS_INDEX,
+             DEFAULT_POEMS_RESULT,
+             false
+             );
+
+        // backup the in-memory database
+        goodStatement( dboConnection, "call syscs_util.syscs_backup_database( '" + BACKUP_DIR2 + "' )" );
+
+        // free up the memory consumed by the in-memory database
+        try {
+            dboConnection = DriverManager.getConnection
+                ( "jdbc:derby:memory:lbt1;user=" + TEST_DBO + ";password=" + dboPassword + ";drop=true" );
+            fail( "Expected to get an exception!" );
+        }
+        catch (SQLException se)
+        {
+            assertEquals( GOOD_SHUTDOWN, se.getSQLState() );
+        }
+
+        //
+        // Now restore the second backup and verify it
+        //
+        source = BACKUP_DIR2 + File.separator + "lbt1";
+        dboConnection = DriverManager.getConnection
+            ( "jdbc:derby:memory:lbt2;user=" + TEST_DBO + ";password=" + dboPassword + ";restoreFrom=" + source );
+        ruthConnection = DriverManager.getConnection
+            ( "jdbc:derby:memory:lbt2;user=" + RUTH + ";password=" + ruthPassword );
+        assertResults
+            (
+             ruthConnection,
+             READ_POEMS_INDEX,
+             DEFAULT_POEMS_RESULT,
+             false
+             );
+        
+        // free up the memory consumed by the in-memory database
+        try {
+            dboConnection = DriverManager.getConnection
+                ( "jdbc:derby:memory:lbt2;user=" + TEST_DBO + ";password=" + dboPassword + ";drop=true" );
+            fail( "Expected to get an exception!" );
+        }
+        catch (SQLException se)
+        {
+            assertEquals( GOOD_SHUTDOWN, se.getSQLState() );
+        }
+    }
+
+}

Propchange: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneBackupTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSuite.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSuite.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSuite.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSuite.java Fri May  2 13:55:51 2014
@@ -65,6 +65,7 @@ public class LuceneSuite extends BaseTes
             suite.addTest(LuceneCollationTest.suite());
             suite.addTest(LuceneCoarseAuthorizationTest.suite());
             suite.addTest(LuceneInMemoryTest.suite());
+            suite.addTest(LuceneBackupTest.suite());
         }
 
         return suite;

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSupportPermsTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSupportPermsTest.java?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSupportPermsTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/LuceneSupportPermsTest.java Fri May  2 13:55:51 2014
@@ -31,6 +31,7 @@ import java.sql.SQLException;
 import java.sql.SQLWarning;
 import java.sql.Connection;
 import java.sql.Date;
+import java.sql.DriverManager;
 import java.sql.Statement;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -98,6 +99,7 @@ public class LuceneSupportPermsTest exte
 	private static  final   String      BAD_DIRECTORY                      = "42XBI";
 	private static  final   String      BAD_COLUMN_NAME                 = "42XBJ";
     private static  final   String      NONEXISTENT_TABLE_FUNCTION  ="42ZB4";
+    private static  final   String      INCOMPATIBLE_ENCRYPTION = "42XBL";
 
     private static  final   String      POLICY_FILE = "org/apache/derbyTesting/functionTests/tests/lang/luceneSupport.policy";
 
@@ -968,6 +970,48 @@ public class LuceneSupportPermsTest exte
              );
     }
 
+   /**
+     * <p>
+     * Test that encryption and the Lucene plugin are incompatible.
+     * </p>
+     */
+    public  void    test_010_encryption()
+        throws Exception
+    {
+        Connection  dboConnection = openUserConnection( TEST_DBO );
+        String      password = getTestConfiguration().getPassword( TEST_DBO );
+        String      encryptDatabaseURL = "jdbc:derby:" + DB_NAME + ";user=" + TEST_DBO + ";password=" + password +
+                ";dataEncryption=true;bootPassword=tryToGuessThis";
+        String      decryptDatabaseURL = "jdbc:derby:" + DB_NAME + ";user=" + TEST_DBO + ";password=" + password +
+                ";decryptDatabase=true;bootPassword=tryToGuessThis";
+
+        goodStatement( dboConnection, LOAD_TOOL );
+        getTestConfiguration().shutdownDatabase();
+
+        // verify that you can't encrypt the database now
+        try {
+            DriverManager.getConnection( encryptDatabaseURL );
+            fail( "Should not have been able to get a connection!" );
+        }
+        catch (SQLException se)
+        {
+            assertSQLState( INCOMPATIBLE_ENCRYPTION, se );
+        }
+
+        // now unload the tool and encrypt the database
+        dboConnection = openUserConnection( TEST_DBO );
+        goodStatement( dboConnection, UNLOAD_TOOL );
+        getTestConfiguration().shutdownDatabase();
+        dboConnection = DriverManager.getConnection( encryptDatabaseURL );
+
+        // now you can't load the plugin
+        expectExecutionError( dboConnection, INCOMPATIBLE_ENCRYPTION, LOAD_TOOL );
+
+        // turn off encryption
+        getTestConfiguration().shutdownDatabase();
+        dboConnection = DriverManager.getConnection( decryptDatabaseURL );
+    }
+    
     ///////////////////////////////////////////////////////////////////////////////////
     //
     // MINIONS

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/luceneSupport.policy
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/luceneSupport.policy?rev=1591910&r1=1591909&r2=1591910&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/luceneSupport.policy (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/luceneSupport.policy Fri May  2 13:55:51 2014
@@ -454,8 +454,10 @@ grant codeBase "${derbyTesting.codejar}d
   permission java.util.PropertyPermission "derby.system.home", "read";
 
   // all databases under derby.system.home 
-  permission java.io.FilePermission "${derby.system.home}${/}lucenesupportpermsdb${/}lucene", "read,write,delete";
-  permission java.io.FilePermission "${derby.system.home}${/}lucenesupportpermsdb${/}lucene${/}-", "read,write,delete";
+  permission java.io.FilePermission "${derby.system.home}${/}lucenesupportpermsdb${/}LUCENE", "read,write,delete";
+  permission java.io.FilePermission "${derby.system.home}${/}lucenesupportpermsdb${/}LUCENE${/}-", "read,write,delete";
+  permission java.io.FilePermission "${derby.system.home}${/}lucenebackupdb${/}LUCENE", "read,write,delete";
+  permission java.io.FilePermission "${derby.system.home}${/}lucenebackupdb${/}LUCENE${/}-", "read,write,delete";
 
   permission java.io.FilePermission "${derbyTesting.lucene.core.jar.file}", "read";
   permission java.util.PropertyPermission "user.dir", "read";
@@ -468,8 +470,10 @@ grant codeBase "${derbyTesting.codejar}d
 grant codeBase "${derbyTesting.lucene.core}"
 {
   // permissions for file access, write access only to sandbox:
-  permission java.io.FilePermission "${derby.system.home}${/}lucenesupportpermsdb${/}lucene", "read,write,delete";
-  permission java.io.FilePermission "${derby.system.home}${/}lucenesupportpermsdb${/}lucene${/}-", "read,write,delete";
+  permission java.io.FilePermission "${derby.system.home}${/}lucenesupportpermsdb${/}LUCENE", "read,write,delete";
+  permission java.io.FilePermission "${derby.system.home}${/}lucenesupportpermsdb${/}LUCENE${/}-", "read,write,delete";
+  permission java.io.FilePermission "${derby.system.home}${/}lucenebackupdb${/}LUCENE", "read,write,delete";
+  permission java.io.FilePermission "${derby.system.home}${/}lucenebackupdb${/}LUCENE${/}-", "read,write,delete";
   
   // Basic permissions needed for Lucene to work:
   permission java.util.PropertyPermission "user.dir", "read";