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 ka...@apache.org on 2007/05/27 14:57:07 UTC

svn commit: r541981 - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/store/raw/data/ engine/org/apache/derby/impl/jdbc/ testing/org/apache/derbyTesting/functionTests/tests/jdbc4/ testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/

Author: kahatlen
Date: Sun May 27 05:57:05 2007
New Revision: 541981

URL: http://svn.apache.org/viewvc?view=rev&rev=541981
Log:
DERBY-2379: Encrypt temporary files for LOBs if database is encrypted

Contributed by Anurag Shekhar.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EncryptedLOBFile.java   (with props)
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBFile.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobStreamControl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobUtf8Writer.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedBlob.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedClob.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBInputStream.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBOutputStream.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBStreamControl.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbc4/LobStreamTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/BlobClob4BlobTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/ClobUpdateableReaderTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/LobStreamsTest.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java Sun May 27 05:57:05 2007
@@ -373,4 +373,10 @@
     public StorageFactory getStorageFactory();
 
 	public void	stop();
+
+    /**
+     * Returns if data base is in encrypted mode.
+     * @return true if database encrypted false otherwise
+     */
+    public boolean databaseEncrypted();
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobStreamControl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobStreamControl.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobStreamControl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobStreamControl.java Sun May 27 05:57:05 2007
@@ -31,6 +31,7 @@
 import java.io.UTFDataFormatException;
 import java.io.Writer;
 import java.sql.SQLException;
+import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.util.ByteArray;
 
@@ -193,9 +194,11 @@
      * @param pos byte postion
      * @return current byte postion
      * @throws IOException
+     * @throws StandardException
+     * @throws SQLException
      */
     synchronized long insertString (String str, long pos)
-                                            throws IOException, SQLException {
+                 throws IOException, SQLException, StandardException {
         int len = str.length();
         if (pos == super.getLength()) {
             byte b [] = getByteFromString (str);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobUtf8Writer.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobUtf8Writer.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobUtf8Writer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/ClobUtf8Writer.java Sun May 27 05:57:05 2007
@@ -27,6 +27,7 @@
 import java.io.OutputStream;
 import java.io.Writer;
 import java.sql.SQLException;
+import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.services.i18n.MessageService;
 
@@ -106,6 +107,9 @@
             IOException ioe = new IOException (e.getMessage());
             ioe.initCause (e);
             throw ioe;
+        }
+        catch (StandardException se) {
+            throw new IOException (se.getMessage());
         }
     }
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedBlob.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedBlob.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedBlob.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedBlob.java Sun May 27 05:57:05 2007
@@ -119,6 +119,9 @@
          catch (IOException e) {
              throw Util.setStreamFailure (e);
          }
+         catch (StandardException se) {
+            throw Util.generateCsSQLException (se);
+         }
      }
      
     /*
@@ -228,10 +231,15 @@
         int c;
         if (materialized)
         {
-            if (pos >= control.getLength())
-                return -1;
-            else
-                c = control.read (pos);
+            try {
+                if (pos >= control.getLength())
+                    return -1;
+                else
+                    c = control.read (pos);
+            }
+            catch (StandardException se) {
+                throw Util.generateCsSQLException (se);
+            }
         }
         else
             c = biStream.read();
@@ -784,6 +792,9 @@
         catch (IOException e) {
             throw Util.setStreamFailure (e);
         }
+        catch (StandardException se) {
+            throw Util.generateCsSQLException (se);
+        }
     }
 
    /**
@@ -822,6 +833,9 @@
             catch (IOException e) {
                 throw Util.setStreamFailure (e);
             }
+            catch (StandardException se) {
+                throw Util.generateCsSQLException (se);
+            }
 	}
 
 	/**
@@ -835,7 +849,7 @@
     * @exception SQLException Feature not implemented for now.
 	*/
 	public void truncate(long len)
-    throws SQLException
+                                        throws SQLException
 	{
             if (len > length())
                 throw Util.generateCsSQLException(
@@ -853,6 +867,9 @@
             }
             catch (IOException e) {
                 throw Util.setStreamFailure (e);
+            }
+            catch (StandardException se) {
+                throw Util.generateCsSQLException (se);
             }
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedClob.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedClob.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedClob.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EmbedClob.java Sun May 27 05:57:05 2007
@@ -99,10 +99,12 @@
         try {
             control.insertString (clobString, 0);
         }
-
         catch (IOException e) {
             throw Util.setStreamFailure (e);
         }
+        catch (StandardException se) {
+            throw Util.generateCsSQLException (se);
+        }
     }
 
     /**
@@ -720,6 +722,9 @@
             } catch (IOException e) {
                 throw Util.setStreamFailure(e);
             }
+            catch (StandardException se) {
+                throw Util.generateCsSQLException (se);
+            }
         }
 
 	/**
@@ -759,6 +764,8 @@
             return control.getWriter(pos - 1);
         } catch (IOException e) {
             throw Util.setStreamFailure(e);
+        } catch (StandardException se) {
+            throw Util.generateCsSQLException (se);
         }
     }
 

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EncryptedLOBFile.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EncryptedLOBFile.java?view=auto&rev=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EncryptedLOBFile.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EncryptedLOBFile.java Sun May 27 05:57:05 2007
@@ -0,0 +1,358 @@
+/*
+
+   Derby - Class org.apache.derby.impl.jdbc.EncryptedLOBFile
+
+   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.derby.impl.jdbc;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.services.i18n.MessageService;
+import org.apache.derby.iapi.store.raw.data.DataFactory;
+import org.apache.derby.io.StorageFile;
+
+/**
+ * This class is a wrapper class on top of StorageRandomAccess to provide common
+ * methods to write in encrypted file.
+ * This class is NOT thread safe. The user class should take care
+ * of synchronization if being used in multi threaded environment.
+ */
+class EncryptedLOBFile extends LOBFile {
+    /** Block size for encryption. */
+    private final int blockSize;
+    /** Leftover bytes. Stored in memory until they fill one block .*/
+    private final byte [] tail;
+    /** Number of actual bytes in tail array. */
+    private int tailSize;
+    /** Current file position. */
+    private long currentPos;
+    /** Factory object used for encryption and decryption. */
+    private final DataFactory df;
+
+    /**
+     * Constructs the EncryptedLOBFile object with encryption support.
+     *
+     * @param lobFile StorageFile Object for which file will be created
+     * @param df data factory for encryption and decription
+     * @throws FileNotFoundException if the file exists but is a directory or
+     * cannot be opened
+     */
+    EncryptedLOBFile(StorageFile lobFile, DataFactory df)
+                                                throws FileNotFoundException {
+        super(lobFile);
+        this.df = df;
+        blockSize = df.getEncryptionBlockSize();
+        tail = new byte [blockSize];
+        tailSize = 0;
+    }
+
+    /**
+     * Find the blocks containing the data we are interested in.
+     *
+     * @param pos first position we are interested in
+     * @param len number of bytes of interest
+     * @return byte array containing all the blocks of data the specified
+     * region spans over
+     */
+    private byte [] getBlocks (long pos, int len)
+                                        throws IOException, StandardException {
+        if (len < 0)
+            throw new IndexOutOfBoundsException (
+                    MessageService.getTextMessage (
+                        SQLState.BLOB_NONPOSITIVE_LENGTH, new Integer (len)));
+        //starting position of the 1st block
+        long startPos = pos - pos % blockSize;
+        //end position of last block
+        long endPos = (pos + len + blockSize - 1) / blockSize * blockSize;
+
+        byte [] data = new byte [(int) (endPos - startPos)];
+        super.seek (startPos);
+        super.read (data, 0, data.length);
+        return data;
+    }
+
+    /**
+     * Returns file length.
+     * @return file length
+     * @throws IOException if an I/O error occurs
+     */
+    long length() throws IOException {
+        return super.length() + tailSize;
+    }
+
+    /**
+     * Returns the currrent position in the file.
+     * @return current position of file pointer
+     */
+    long getFilePointer() {
+        return currentPos;
+    }
+
+    /**
+     * Sets the current file pointer to specific location.
+     * @param pos new position
+     * @throws IOException
+     */
+    void seek (long pos) throws IOException {
+        if (pos > length()) {
+            //this should never happen
+            //this exception will mean internal error most
+            //probably in LOBStreamControl
+            throw new IllegalArgumentException ("Internal Error");
+        }
+        if (pos < super.length()) {
+            super.seek (pos);
+        }
+        currentPos = pos;
+    }
+
+    /**
+     * Writes one byte into the file.
+     * @param b byte value
+     * @throws IOException if disk operation fails
+     * @throws StandardException if error occured during encryption/decryption
+     */
+    void write (int b) throws IOException, StandardException {
+        long length = super.length();
+        if (currentPos >= length) {
+            //current position is in memory
+            int pos = (int) (currentPos - length);
+            tail [pos] = (byte) b;
+            if (pos >= tailSize) {
+                tailSize = pos + 1;
+            }
+            if (tailSize == blockSize) {
+                //we have enough data to fill one block encrypt and write
+                //in file
+                byte [] cypherText = new byte [blockSize];
+                df.encrypt (tail, 0, tailSize, cypherText, 0, false);
+                super.seek (length);
+                super.write (cypherText);
+                tailSize = 0;
+            }
+        }
+        else {
+            //write position is in the middle of the file
+            //get the complete block in which the destination byte falls into
+            byte [] cypherText = getBlocks (currentPos, 1);
+            byte [] clearText = new byte [blockSize];
+            //decrypt the block before updating
+            df.decrypt(cypherText, 0, blockSize, clearText, 0);
+            clearText [(int) (currentPos%blockSize)] = (byte) b;
+            //encrypt and write back
+            df.encrypt (clearText, 0, blockSize, cypherText, 0, false);
+            super.seek (currentPos - currentPos % blockSize);
+            super.write (cypherText);
+        }
+        currentPos++;
+    }
+
+    /**
+     * Writes length number of bytes from buffer starting from off position.
+     * @param b byte array containing bytes to be written
+     * @param off starting offset of the byte array from where the
+     * data should be written to the file
+     * @param len number of bytes to be written
+     * @throws IOException if disk operation fails
+     * @throws StandardException if error occured during encryption/decryption
+     */
+    void write(byte[] b, int off, int len)
+                                    throws IOException, StandardException {
+        long fileLength = super.length();
+        if (currentPos < fileLength) {
+            //starting position for write is in file
+            //find out if we need to update memory
+            int overFlow = (int) Math.max(0L, currentPos + len - fileLength);
+            long oldPos = currentPos;
+            //get the block containing bytes we are going to overwrite
+            byte [] cypherText = getBlocks (currentPos, len - overFlow);
+            byte [] clearText = new byte [cypherText.length];
+            //decrypt the data before updating
+            for (int i = 0; i < cypherText.length / blockSize; i++)
+                df.decrypt (cypherText, i * blockSize, blockSize, clearText,
+                                                                i * blockSize);
+            //update the data
+            System.arraycopy (b, off, clearText, (int) (currentPos%blockSize),
+                len - overFlow);
+            //encrypt and write back
+            for (int i = 0; i < cypherText.length / blockSize; i++)
+                df.encrypt (clearText, i * blockSize, blockSize,
+                                        cypherText, i * blockSize, false);
+            super.seek (oldPos - oldPos % blockSize);
+            super.write (cypherText);
+            currentPos = oldPos + cypherText.length;
+            //nothing to keep in memory.
+            if (overFlow == 0)
+                return;
+            //adjust the value to perform rest of the writes in tail buffer
+            off = off + len - overFlow;
+            len = overFlow;
+            //write rest of the data in memory
+            currentPos = fileLength;
+        }
+        //starting position in array
+        int pos = (int) (currentPos - fileLength);
+        int finalPos = pos + len;
+        if (finalPos < blockSize) {
+            //updated size won't be enough to perform encryption
+            System.arraycopy (b, off, tail, pos, len);
+            tailSize = Math.max(tailSize, pos + len);
+            currentPos += len;
+            return;
+        }
+        //number of bytes which can be encrypted
+        int encLength = finalPos - finalPos % blockSize;
+        int leftOver = finalPos % blockSize;
+        byte [] clearText = new byte [encLength];
+        //create array to encrypt
+        //copy the bytes from tail which won't be overwritten
+        System.arraycopy (tail, 0, clearText, 0, pos);
+        //copy remaining data into array
+        System.arraycopy (b, off, clearText, pos, encLength - pos);
+        byte [] cypherText = new byte [clearText.length];
+        //encrypt and write
+        for (int offset = 0; offset < cypherText.length ; offset += blockSize)
+            df.encrypt (clearText, offset, blockSize, cypherText,
+                                                        offset, false);
+        super.seek (fileLength);
+        super.write (cypherText);
+        //copy rest of it in tail
+        System.arraycopy (b, off + len - leftOver, tail, 0, leftOver);
+        tailSize = leftOver;
+        currentPos = tailSize + fileLength + cypherText.length;
+    }
+
+    /**
+     * Write the buffer into file at current position. It overwrites the
+     * data if current position is in the middle of the file and appends into
+     * the file if the total length exceeds the file size.
+     * @param b byte array to be written
+     * @throws IOException if disk operation fails
+     * @throws StandardException if error occured during encryption/decryption
+     */
+    void write(byte[] b) throws IOException, StandardException {
+        write (b, 0, b.length);
+    }
+
+    /**
+     * closes the file.
+     * @throws IOException
+     */
+    void close() throws IOException {
+        super.close();
+    }
+
+    /**
+     * Reads one byte from file.
+     * @return byte
+     * @throws IOException if disk operation fails
+     * @throws StandardException if error occured during encryption/decryption
+     */
+    int readByte() throws IOException, StandardException {
+        if (currentPos >= length())
+            throw new EOFException ();
+        long fileLength = super.length();
+        if (currentPos >= fileLength)
+            return tail [(int) (currentPos++ - fileLength)] & 0xff;
+        //get the block containing the byte we are interested in
+        byte cypherText [] = getBlocks (currentPos, 1);
+        byte [] clearText = new byte [cypherText.length];
+        df.decrypt (cypherText, 0, cypherText.length, clearText, 0);
+        return clearText [(int) (currentPos++ % blockSize)] & 0xff;
+    }
+
+    /**
+     * Reads len or remaining bytes in the file (whichever is lower) bytes
+     * into buff starting from off position of the buffer.
+     * @param buff byte array to fill read bytes
+     * @param off offset of buff where the byte will be written
+     * @param len number of bytes to be read
+     * @return number of bytes read
+     * @throws IOException if disk operation fails
+     * @throws StandardException if error occured during encryption/decryption
+     */
+    int read(byte[] buff, int off, int len)
+                                        throws IOException, StandardException {
+        long fileLength = super.length();
+        if (currentPos < fileLength) {
+            //starting position is in file
+            //find number of bytes spilling out of file
+            int overFlow = (int) Math.max(0L, currentPos + len - fileLength);
+            //get all the blocks
+            byte [] cypherText = getBlocks (currentPos, len - overFlow);
+            byte [] tmpByte = new byte [cypherText.length];
+            //decrypt
+            for (int offset = 0; offset < cypherText.length; offset += blockSize) {
+                df.decrypt (cypherText, offset, blockSize, tmpByte,
+                                                                offset);
+            }
+            //copy the bytes we are interested in
+            System.arraycopy (tmpByte, (int) (currentPos%blockSize), buff,
+                                                        off, len - overFlow);
+            if (overFlow == 0) {
+                currentPos += len;
+                return len;
+            }
+            //find out total number of bytes we can read
+            int newLen = Math.min(overFlow, tailSize);
+            //fill the buffer from tail
+            System.arraycopy (tail, 0, buff, off + len - overFlow, newLen);
+            currentPos += len - overFlow + newLen;
+            return len - overFlow + newLen;
+        }
+        int newLen = (int) Math.min (
+            tailSize - currentPos + fileLength, len);
+        if (newLen == 0 && len != 0)
+            return -1;
+
+        System.arraycopy (tail, (int) (currentPos - fileLength),
+                            buff, off, newLen);
+        currentPos += newLen;
+        return newLen;
+    }
+
+    /**
+     * Sets the file length to a given size. If the new size is smaller than the
+     * file length the file is truncated.
+     * @param size new  file size. Must be lower than file length.
+     * @throws IOException if file i/o fails
+     * @throws StandardException if error occured during encryption/decryption
+     */
+    void setLength(long size) throws IOException, StandardException {
+        if (size > length()) {
+            //this should never happen
+            //this exception will mean internal error most
+            //probably in LOBStreamControl
+            throw new IllegalArgumentException ("Internal Error");
+        }
+        long length = super.length();
+        if (size < length) {
+            byte [] block = getBlocks (size, 1);
+            super.setLength (size - size % blockSize);
+            df.decrypt (block, 0, blockSize, tail, 0);
+            tailSize = (int) (size % blockSize);
+        }
+        else {
+            tailSize = (int) (size - length);
+        }
+    }
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/EncryptedLOBFile.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBFile.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBFile.java?view=auto&rev=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBFile.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBFile.java Sun May 27 05:57:05 2007
@@ -0,0 +1,158 @@
+/*
+
+   Derby - Class org.apache.derby.impl.jdbc.LOBFile
+
+   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.derby.impl.jdbc;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.io.StorageFile;
+import org.apache.derby.io.StorageRandomAccessFile;
+
+/**
+ * LOBFile is a wrapper over StorageRandomAccessFile. The purpose of this class
+ * is to let the user of this class access StorageRandomAccessFile in plain and
+ * in encrypted for without having to change code.
+ */
+class LOBFile {
+    private final StorageRandomAccessFile randomAccessFile;
+    /**
+     * Constructs LOBFile.
+     *
+     * @param lobFile StorageFile object for which the file will be created
+     * @throws FileNotFoundException if the file exists but is a directory or
+     * cannot be opened
+     */
+    LOBFile(StorageFile lobFile) throws FileNotFoundException {
+        randomAccessFile = lobFile.getRandomAccessFile("rw");
+    }
+
+    /**
+     * Returns length of the file.
+     * @return length of the file
+     * @throws IOException if an I/O error occurs
+     */
+    long length() throws IOException {
+        return randomAccessFile.length();
+    }
+
+    /**
+     * Sets the file pointer to a given position.
+     * @param pos new position
+     * @throws IOException if an I/O error occurs
+     */
+    void seek(long pos) throws IOException {
+        randomAccessFile.seek (pos);
+    }
+
+    /**
+     * Writes one bytes into the file.
+     * @param b int value of the byte
+     * @throws IOException if an I/O error occurs
+     * @throws StandardException it won't be thrown, it's in signature to allow
+     *              subclasses to throw StandardException
+     */
+    void write(int b) throws IOException, StandardException {
+        randomAccessFile.write (b);
+    }
+
+    /**
+     * Returns the current position of the file pointer.
+     * @return file pointer
+     * @throws IOException if an I/O error occurs
+     */
+    long getFilePointer() throws IOException {
+        return randomAccessFile.getFilePointer();
+    }
+
+    /**
+     * Writes a segment of bytes into the file.
+     * @param b byte array containing bytes to write into the file
+     * @param off starting position of segment
+     * @param len number of bytes to be written
+     * @throws IOException if an I/O error occurs
+     * @throws StandardException it won't be thrown, it's in signature to allow
+     *              subclasses to throw StandardException
+     */
+    void write(byte[] b, int off, int len)
+                                    throws IOException, StandardException {
+            randomAccessFile.write (b, off, len);
+    }
+
+    /**
+     * Reads one byte from file.
+     * @return byte
+     * @throws IOException if disk operation fails
+     * @throws StandardException it won't be thrown, it's in signature to allow
+     *              subclasses to throw StandardException
+     */
+    int readByte() throws IOException, StandardException {
+        return randomAccessFile.readByte();
+    }
+
+    /**
+     * Reads len number of bytes from the file starting from off position
+     * in the buffer.
+     * @param buff buffer
+     * @param off starting position of buffer
+     * @param len number of bytes
+     * @return number of bytes read
+     * @throws IOException if an I/O error occurs
+     * @throws StandardException it won't be thrown, it's in signature to allow
+     *              subclasses to throw StandardException
+     */
+    int read(byte[] buff, int off, int len)
+                                    throws IOException, StandardException {
+        return randomAccessFile.read (buff, off, len);
+    }
+
+    /**
+     * Closes the file.
+     * @throws IOException if an I/O error occurs
+     * @throws StandardException it won't be thrown, it's in signature to allow
+     *              subclasses to throw StandardException
+     */
+    void close() throws IOException {
+        randomAccessFile.close();
+    }
+
+    /**
+     * Sets the file length to a given size.
+     * @param size new size
+     * @throws IOException if an I/O error occurs
+     * @throws StandardException it won't be thrown, it's in signature to allow
+     *              subclasses to throw StandardException
+     */
+    void setLength(long size) throws IOException, StandardException {
+        randomAccessFile.setLength (size);
+    }
+
+    /**
+     * Writes a buffer completely into the file.
+     * @param buf buffer to write
+     * @throws IOException if an I/O error occurs
+     * @throws StandardException it won't be thrown, it's in signature to allow
+     *              subclasses to throw StandardException
+     */
+    void write(byte[] buf) throws IOException, StandardException {
+        randomAccessFile.write (buf);
+    }
+
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBFile.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBInputStream.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBInputStream.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBInputStream.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBInputStream.java Sun May 27 05:57:05 2007
@@ -25,6 +25,7 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.sql.SQLException;
+import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.services.i18n.MessageService;
 import org.apache.derby.shared.common.error.ExceptionUtil;
@@ -125,6 +126,9 @@
         } catch (SQLException e) {
             return handleSQLException (e);
         }
+        catch (StandardException se) {
+            throw new IOException (se.getMessage());
+        }
     }
 
     /**
@@ -165,6 +169,8 @@
             return ret;
         } catch (SQLException e) {
             throw new IOException(e.getMessage());
+        } catch (StandardException se) {
+            throw new IOException (se.getMessage());
         }
     }
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBOutputStream.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBOutputStream.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBOutputStream.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBOutputStream.java Sun May 27 05:57:05 2007
@@ -26,6 +26,7 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.sql.SQLException;
+import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.services.i18n.MessageService;
 import org.apache.derby.shared.common.error.ExceptionUtil;
@@ -70,6 +71,8 @@
             pos = control.write(b, pos);
         } catch (SQLException e) {
             throw  new IOException(e.getMessage());
+        } catch (StandardException se) {
+            throw new IOException (se.getMessage());
         }
     }
 
@@ -114,6 +117,8 @@
                                   SQLState.BLOB_INVALID_OFFSET)))
                     throw new ArrayIndexOutOfBoundsException (e.getMessage());
             throw new IOException(e.getMessage());
+        } catch (StandardException se) {
+            throw new IOException (se.getMessage());
         }
     }
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBStreamControl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBStreamControl.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBStreamControl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/jdbc/LOBStreamControl.java Sun May 27 05:57:05 2007
@@ -55,7 +55,7 @@
  */
 
 class LOBStreamControl {
-    private StorageRandomAccessFile tmpFile;
+    private LOBFile tmpFile;
     private StorageFile lobFile;
     private byte [] dataBytes = new byte [0];
     private boolean isBytes = true;
@@ -70,7 +70,8 @@
         updateCount = 0;
     }
 
-    private void init(byte [] b, long len) throws IOException, SQLException {
+    private void init(byte [] b, long len)
+                    throws IOException, SQLException, StandardException {
         try {
             AccessController.doPrivileged (new PrivilegedExceptionAction() {
                 public Object run() throws IOException, StandardException {
@@ -81,7 +82,11 @@
                     //create a temporary file
                     lobFile =
                         df.getStorageFactory().createTemporaryFile("lob", null);
-                    tmpFile = lobFile.getRandomAccessFile ("rw");
+                    if (df.databaseEncrypted()) {
+                        tmpFile = new EncryptedLOBFile (lobFile, df);
+                    }
+                    else
+                        tmpFile = new LOBFile (lobFile);
                     return null;
                 }
             });
@@ -136,7 +141,8 @@
         }
     }
 
-    private void isValidPostion(long pos) throws SQLException, IOException {
+    private void isValidPostion(long pos)
+                        throws SQLException, IOException {
         if (pos < 0)
             throw Util.generateCsSQLException(
                     SQLState.BLOB_NONPOSITIVE_LENGTH, new Long(pos + 1));
@@ -170,9 +176,10 @@
      * @param b byte
      * @param pos
      * @return new postion
-     * @throws IOException, SQLException
+     * @throws IOException, SQLException, StandardException
      */
-    synchronized long write(int b, long pos) throws IOException, SQLException {
+    synchronized long write(int b, long pos)
+                throws IOException, SQLException, StandardException {
         isValidPostion(pos);
         updateCount++;
         if (isBytes) {
@@ -196,10 +203,10 @@
      * @param len number of bytes to be copied
      * @param pos starting postion
      * @return new postion
-     * @throws IOException, SQLException
+     * @throws IOException, SQLException, StandardException
      */
     synchronized long write(byte[] b, int off, int len, long pos)
-    throws IOException, SQLException {
+                        throws IOException, SQLException, StandardException {
         try {
             isValidPostion(pos);
             isValidOffset(off, b.length);
@@ -227,9 +234,10 @@
      * Reads one byte.
      * @param pos postion from where to read
      * @return byte
-     * @throws IOException, SQLException
+     * @throws IOException, SQLException, StandardException
      */
-    synchronized int read(long pos) throws IOException, SQLException {
+    synchronized int read(long pos)
+                throws IOException, SQLException, StandardException {
         isValidPostion(pos);
         if (isBytes) {
             if (dataBytes.length == pos)
@@ -262,10 +270,10 @@
      * @param len number of bytes to read
      * @param pos initial postion before reading
      * @return number new postion
-     * @throws IOException, SQLException
+     * @throws IOException, SQLException, StandardException
      */
     synchronized int read(byte[] buff, int off, int len, long pos)
-    throws IOException, SQLException {
+    throws IOException, SQLException, StandardException {
         isValidPostion(pos);
         isValidOffset(off, buff.length);
         if (isBytes) {
@@ -309,7 +317,8 @@
      * @param size new size should be smaller than exisiting size
      * @throws IOException, SQLException
      */
-    synchronized void truncate(long size) throws IOException, SQLException {
+    synchronized void truncate(long size)
+                        throws IOException, SQLException, StandardException {
         isValidPostion(size);
         if (isBytes) {
             byte [] tmpByte = new byte [(int) size];
@@ -322,8 +331,14 @@
                 isBytes = true;
                 tmpFile.close();
                 tmpFile = null;
-            } else
-                tmpFile.setLength(size);
+            } else {
+                try {
+                    tmpFile.setLength(size);
+                }
+                catch (StandardException se) {
+                    Util.generateCsSQLException (se);
+                }
+            }
         }
     }
 
@@ -331,10 +346,10 @@
      * Copies bytes from stream to local storage.
      * @param inStream
      * @param length length to be copied
-     * @throws IOException, SQLException
+     * @throws IOException, SQLException, StandardException
      */
     synchronized void copyData(InputStream inStream,
-            long length) throws IOException, SQLException {
+            long length) throws IOException, SQLException, StandardException {
         byte [] data = new byte [MAX_BUF_SIZE];
         long sz = 0;
         while (sz < length) {
@@ -388,10 +403,11 @@
      * @param endPos exclusive end position of current block
      * @return Current position after write.
      * @throws IOExcepton if writing to temporary file fails
+     * @throws StandardException
      * @throws SQLException
      */
     synchronized long replaceBytes (byte [] buf, long stPos, long endPos) 
-                                            throws IOException, SQLException {
+                         throws IOException, SQLException, StandardException {
         long length = getLength();
         long finalLength = length - endPos + stPos + buf.length;
         if (isBytes) {
@@ -417,7 +433,7 @@
             //create new file with 0 size
             
             byte tmp [] = new byte [0];
-            StorageRandomAccessFile oldFile = tmpFile;
+            LOBFile oldFile = tmpFile;
             init (tmp, 0);
             byte [] tmpByte = new byte [1024];
             long sz = stPos;

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbc4/LobStreamTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbc4/LobStreamTest.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbc4/LobStreamTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbc4/LobStreamTest.java Sun May 27 05:57:05 2007
@@ -31,6 +31,7 @@
 import java.sql.Blob;
 import java.sql.Connection;
 import org.apache.derbyTesting.junit.BaseJDBCTestCase;
+import org.apache.derbyTesting.junit.Decorator;
 import org.apache.derbyTesting.junit.TestConfiguration;
 
 
@@ -437,7 +438,12 @@
      * Suite method automatically generated by JUnit module.
      */
     public static Test suite() {
-        return TestConfiguration.embeddedSuite(LobStreamTest.class);
+        TestSuite ts  = new TestSuite ("LobStreamTest");
+        ts.addTest(TestConfiguration.embeddedSuite(LobStreamTest.class));
+        TestSuite encSuite = new TestSuite ("LobStreamsTest:encrypted");
+        encSuite.addTestSuite (LobStreamTest.class);
+        ts.addTest(Decorator.encryptedDatabase (encSuite));
+        return ts;
     }
 
 }

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/BlobClob4BlobTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/BlobClob4BlobTest.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/BlobClob4BlobTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/BlobClob4BlobTest.java Sun May 27 05:57:05 2007
@@ -34,6 +34,7 @@
 import org.apache.derbyTesting.junit.BaseJDBCTestCase;
 import org.apache.derbyTesting.junit.CleanDatabaseTestSetup;
 import org.apache.derbyTesting.junit.DatabasePropertyTestSetup;
+import org.apache.derbyTesting.junit.Decorator;
 import org.apache.derbyTesting.junit.Utilities;
 
 import junit.framework.*;
@@ -2767,6 +2768,9 @@
         suite.addTest(
                 TestConfiguration.clientServerSuite(BlobClob4BlobTest.class));
 
+        TestSuite encSuite = new TestSuite ("BlobClob4BlobTest:encrypted");
+        encSuite.addTestSuite (BlobClob4BlobTest.class);
+        suite.addTest(Decorator.encryptedDatabase (encSuite));
         return new CleanDatabaseTestSetup(
                 DatabasePropertyTestSetup.setLockTimeouts(suite, 2, 4));
     }

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/ClobUpdateableReaderTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/ClobUpdateableReaderTest.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/ClobUpdateableReaderTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/ClobUpdateableReaderTest.java Sun May 27 05:57:05 2007
@@ -29,7 +29,9 @@
 import java.sql.ResultSet;
 import java.sql.Statement;
 import junit.framework.Test;
+import junit.framework.TestSuite;
 import org.apache.derbyTesting.junit.BaseJDBCTestCase;
+import org.apache.derbyTesting.junit.Decorator;
 import org.apache.derbyTesting.junit.TestConfiguration;
 
 /**
@@ -159,8 +161,13 @@
     }
     
     public static Test suite() {
-        return TestConfiguration.embeddedSuite(
-                    ClobUpdateableReaderTest.class);
+        TestSuite ts = new TestSuite ("ClobUpdateableReaderTest");
+        ts.addTest(TestConfiguration.embeddedSuite(
+                    ClobUpdateableReaderTest.class));
+        TestSuite encSuite = new TestSuite ("ClobUpdateableReaderTest:encrypted");
+        encSuite.addTestSuite (ClobUpdateableReaderTest.class);
+        ts.addTest(Decorator.encryptedDatabase (encSuite));
+        return ts;
     }        
 
     /**

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/LobStreamsTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/LobStreamsTest.java?view=diff&rev=541981&r1=541980&r2=541981
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/LobStreamsTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/LobStreamsTest.java Sun May 27 05:57:05 2007
@@ -37,6 +37,7 @@
 
 import org.apache.derbyTesting.functionTests.util.streams.LoopingAlphabetStream;
 import org.apache.derbyTesting.junit.BaseJDBCTestCase;
+import org.apache.derbyTesting.junit.Decorator;
 import org.apache.derbyTesting.junit.TestConfiguration;
 
 public class LobStreamsTest extends BaseJDBCTestCase {
@@ -443,9 +444,13 @@
      */
     public static Test suite() {
                 
-        return TestConfiguration.defaultSuite (LobStreamsTest.class);
+        TestSuite ts  = new TestSuite ("LobStreamsTest");
+        ts.addTest(TestConfiguration.defaultSuite (LobStreamsTest.class));
+        TestSuite encSuite = new TestSuite ("LobStreamsTest:encrypted");
+        encSuite.addTestSuite (LobStreamsTest.class);
+        ts.addTest(Decorator.encryptedDatabase (encSuite));
+        return ts;
     }
-
     //method to ensure that buffer is filled if there is any data in stream
     private int readBytesFromStream (byte [] b, InputStream is) 
                                                           throws IOException {