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 2007/04/28 02:08:15 UTC

svn commit: r533268 - in /db/derby/code/trunk/java/client/org/apache/derby/client: am/Blob.java am/BlobLocatorInputStream.java am/BlobLocatorOutputStream.java am/Cursor.java am/Lob.java net/NetCursor.java net/NetStatementRequest.java

Author: rhillegas
Date: Fri Apr 27 17:08:14 2007
New Revision: 533268

URL: http://svn.apache.org/viewvc?view=rev&rev=533268
Log:
DERBY-2496: Commit Oystein's blob_v2.diff patch, adding locator support for Blob methods and getXXX() methods.

Added:
    db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorInputStream.java   (with props)
    db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorOutputStream.java   (with props)
Modified:
    db/derby/code/trunk/java/client/org/apache/derby/client/am/Blob.java
    db/derby/code/trunk/java/client/org/apache/derby/client/am/Cursor.java
    db/derby/code/trunk/java/client/org/apache/derby/client/am/Lob.java
    db/derby/code/trunk/java/client/org/apache/derby/client/net/NetCursor.java
    db/derby/code/trunk/java/client/org/apache/derby/client/net/NetStatementRequest.java

Modified: db/derby/code/trunk/java/client/org/apache/derby/client/am/Blob.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/client/org/apache/derby/client/am/Blob.java?view=diff&rev=533268&r1=533267&r2=533268
==============================================================================
--- db/derby/code/trunk/java/client/org/apache/derby/client/am/Blob.java (original)
+++ db/derby/code/trunk/java/client/org/apache/derby/client/am/Blob.java Fri Apr 27 17:08:14 2007
@@ -23,6 +23,7 @@
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.sql.SQLException;
 import java.util.ArrayList;
 
@@ -99,6 +100,19 @@
         binaryStream_ = binaryStream;
         dataType_ |= BINARY_STREAM;
     }
+    
+    /**
+     * Create a <code>Blob</code> object for a Blob value stored 
+     * on the server and indentified by <code>locator</code>.
+     * @param agent context for this Blob object (incl. connection)
+     * @param locator reference id to Blob value on server
+     */
+    public Blob(Agent agent, int locator)  
+    {
+        super(agent, false);
+        locator_ = locator;
+        dataType_ |= LOCATOR;
+    }
 
     // ---------------------------jdbc 2------------------------------------------
 
@@ -127,6 +141,20 @@
             throw se.getSQLException();
         }
     }
+    
+    /**
+     * Get the length in bytes of the <code>Blob</code> value represented by 
+     * this locator based <Blob> object.  
+     * 
+     * A stored procedure call will be made to get it from the server.
+     * @throws org.apache.derby.client.am.SqlException 
+     * @return length of Blob in bytes
+     */
+    long getLocatorLength() throws SqlException
+    {
+        return agent_.connection_.locatorProcedureCall()
+            .blobGetLength(locator_);
+    }
 
   /**
    * Returns as an array of bytes part or all of the <code>BLOB</code>
@@ -196,8 +224,15 @@
         // actual length is the lesser of the number of bytes requested
         // and the number of bytes available from pos to the end
         actualLength = Math.min(sqlLength() - pos + 1, (long) length);
-        byte[] retVal = new byte[(int) actualLength];
-        System.arraycopy(binaryString_, (int) pos + dataOffset_ - 1, retVal, 0, (int) actualLength);
+        byte[] retVal; 
+        if (isLocator()) {
+            retVal = agent_.connection_.locatorProcedureCall()
+                .blobGetBytes(locator_, pos, (int )actualLength);
+        } else {
+            retVal = new byte[(int) actualLength];
+            System.arraycopy(binaryString_, (int) pos + dataOffset_ - 1, 
+                             retVal, 0, (int) actualLength);
+        }
         return retVal;
     }
 
@@ -231,9 +266,12 @@
         if (isBinaryStream())    // this Lob is used for input
         {
             return binaryStream_;
+        } else if (isLocator()) {
+            return new BlobLocatorInputStream(agent_.connection_, this);
+        } else {  // binary string
+            return new java.io.ByteArrayInputStream(binaryString_, dataOffset_,
+                                           binaryString_.length - dataOffset_);
         }
-
-        return new java.io.ByteArrayInputStream(binaryString_, dataOffset_, binaryString_.length - dataOffset_);
     }
 
     public long position(byte[] pattern, long start) throws SQLException {
@@ -271,7 +309,12 @@
     private long positionX(byte[] pattern, long start) throws SqlException {
         checkForClosedConnection();
 
-        return binaryStringPosition(pattern, start);
+        if (isLocator()) {
+            return agent_.connection_.locatorProcedureCall()
+                .blobGetPositionFromBytes(locator_, pattern, start);
+        } else {
+            return binaryStringPosition(pattern, start);
+        }
     }
 
     public long position(java.sql.Blob pattern, long start) throws SQLException {
@@ -310,9 +353,26 @@
         checkForClosedConnection();
 
         try {
-            return binaryStringPosition(pattern.getBytes(1L, 
-                                                         (int)pattern.length()),
-                                        start);
+            if (isLocator()) {
+                if ((pattern instanceof Blob) 
+                    && ((Blob )pattern).isLocator()) {
+                    // Send locator for pattern to server
+                    return agent_.connection_.locatorProcedureCall()
+                        .blobGetPositionFromLocator(locator_, 
+                                                 ((Blob )pattern).getLocator(),
+                                                 start);
+                } else {
+                    // Convert pattern to byte array before sending to server
+                    return agent_.connection_.locatorProcedureCall()
+                        .blobGetPositionFromBytes(locator_, 
+                                  pattern.getBytes(1L, (int )pattern.length()),
+                                  start);
+                }
+            } else { 
+                return binaryStringPosition(
+                                  pattern.getBytes(1L, (int )pattern.length()),
+                                  start);
+            }
         } catch (java.sql.SQLException e) {
             throw new SqlException(e);
         }
@@ -391,9 +451,10 @@
                     new ClientMessageId(SQLState.BLOB_POSITION_TOO_LARGE), new Long(pos));
         }
         
-        if (pos - 1 > binaryString_.length - dataOffset_) {
+        if (pos - 1 > sqlLength()) {
             throw new SqlException(agent_.logWriter_,
-                    new ClientMessageId(SQLState.BLOB_POSITION_TOO_LARGE), new Long(pos));
+                         new ClientMessageId(SQLState.BLOB_POSITION_TOO_LARGE),
+                         new Long(pos));
         }
         
         if ((offset < 0) || offset > bytes.length )
@@ -410,16 +471,35 @@
         if (len == 0) {
             return 0;
         }
+        
         length = Math.min((bytes.length - offset), len);
-        if ((binaryString_.length - dataOffset_ - (int) pos + 1) < length) {
-            byte newbuf[] = new byte[(int) pos + length + dataOffset_ - 1];
-            System.arraycopy(binaryString_, 0, newbuf, 0, binaryString_.length);
-            binaryString_ = newbuf;
-        }
+        if (isLocator()) {  
+            byte[] ba = bytes;
+            if ((offset > 0) || (length < bytes.length)) { 
+                // Copy the part we will use into a new array
+                ba = new byte[length];
+                System.arraycopy(bytes, offset, ba, 0, length);
+            }
+            agent_.connection_.locatorProcedureCall()
+                .blobSetBytes(locator_, pos, length, ba);
+            if (pos+length-1 > sqlLength()) { // Wrote beyond the old end
+                // Update length
+                setSqlLength(pos + length - 1);
+            } 
+        } else {
+            if ((binaryString_.length - dataOffset_ - (int)pos + 1) < length) {
+                byte newbuf[] = new byte[(int) pos + length + dataOffset_ - 1];
+                System.arraycopy(binaryString_, 0, 
+                                 newbuf, 0, binaryString_.length);
+                binaryString_ = newbuf;
+            }
 
-        System.arraycopy(bytes, offset, binaryString_, (int) pos + dataOffset_ - 1, length);
-        binaryStream_ = new java.io.ByteArrayInputStream(binaryString_);
-        setSqlLength(binaryString_.length - dataOffset_);
+            System.arraycopy(bytes, offset, 
+                             binaryString_, (int) pos + dataOffset_ - 1, 
+                             length);
+            binaryStream_ = new java.io.ByteArrayInputStream(binaryString_);
+            setSqlLength(binaryString_.length - dataOffset_);
+        }
         return length;
     }
 
@@ -427,16 +507,35 @@
         //call checkValidity to exit by throwing a SQLException if
         //the Blob object has been freed by calling free() on it
         checkValidity();
-        synchronized (agent_.connection_) {
-            if (agent_.loggingEnabled()) {
-                agent_.logWriter_.traceEntry(this, "setBinaryStream", (int) pos);
-            }
-            BlobOutputStream outStream = new BlobOutputStream(this, pos);
+        try {
+            synchronized (agent_.connection_) {
+                if (agent_.loggingEnabled()) {
+                    agent_.logWriter_.traceEntry(this, "setBinaryStream", (int) pos);
+                }
+                if (pos < 1) {
+                    throw new SqlException(agent_.logWriter_,
+                            new ClientMessageId(SQLState.BLOB_BAD_POSITION),
+                            new Long(pos));
+                }
+                
+                OutputStream outStream;
+                if (isLocator()) {
+                    outStream = new BlobLocatorOutputStream(agent_.connection_,
+                                                            this,
+                                                            pos);
+                } else {
+                    outStream = new BlobOutputStream(this, pos);
+                }
 
-            if (agent_.loggingEnabled()) {
-                agent_.logWriter_.traceExit(this, "setBinaryStream", outStream);
+                if (agent_.loggingEnabled()) {
+                    agent_.logWriter_.traceExit(this, 
+                                                "setBinaryStream", 
+                                                outStream);
+                }
+                return outStream;
             }
-            return outStream;
+        } catch ( SqlException se ) {
+            throw se.getSQLException();
         }
     }
 
@@ -458,12 +557,20 @@
                 if (len == this.sqlLength()) {
                     return;
                 }
-                long newLength = (int) len + dataOffset_;
-                byte newbuf[] = new byte[(int) len + dataOffset_];
-                System.arraycopy(binaryString_, 0, newbuf, 0, (int) newLength);
-                binaryString_ = newbuf;
-                binaryStream_ = new java.io.ByteArrayInputStream(binaryString_);
-                setSqlLength(binaryString_.length - dataOffset_);
+                if (isLocator()) {
+                    agent_.connection_.locatorProcedureCall()
+                        .blobTruncate(locator_, len);
+                    setSqlLength(len);
+                } else {
+                    long newLength = (int) len + dataOffset_;
+                    byte newbuf[] = new byte[(int) len + dataOffset_];
+                    System.arraycopy(binaryString_, 0, 
+                                     newbuf, 0, (int) newLength);
+                    binaryString_ = newbuf;
+                    binaryStream_ 
+                        = new java.io.ByteArrayInputStream(binaryString_);
+                    setSqlLength(binaryString_.length - dataOffset_);
+                }
             }
         }
         catch ( SqlException se )
@@ -492,17 +599,27 @@
         //now that free has been called the Blob object is no longer
         //valid
         isValid = false;
-        
-        if(isBinaryStream()) {
-            try {
-                binaryStream_.close();
-            }
-            catch(IOException ioe) {
-                throw new SqlException(null, new ClientMessageId(SQLState.IO_ERROR_UPON_LOB_FREE)).getSQLException();
+        try {            
+            synchronized (agent_.connection_) {
+                if (agent_.loggingEnabled()) {
+                    agent_.logWriter_.traceEntry(this, "free");
+                }
+                if (isBinaryStream()) {
+                    try {
+                        binaryStream_.close();
+                    } catch(IOException ioe) {
+                        throw new SqlException(null, new ClientMessageId(
+                                             SQLState.IO_ERROR_UPON_LOB_FREE));
+                    }
+                } else if (isLocator()) {
+                    agent_.connection_.locatorProcedureCall()
+                        .blobReleaseLocator(locator_);
+                } else {
+                    binaryString_ = null;
+                }
             }
-        }
-        else {
-            binaryString_ = null;
+        } catch (SqlException se) {
+            throw se.getSQLException();
         }
     }
 

Added: db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorInputStream.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorInputStream.java?view=auto&rev=533268
==============================================================================
--- db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorInputStream.java (added)
+++ db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorInputStream.java Fri Apr 27 17:08:14 2007
@@ -0,0 +1,141 @@
+/*
+ 
+   Derby - Class org.apache.derby.client.am.BlobLocatorInputStream
+ 
+   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.client.am;
+
+import java.sql.CallableStatement;
+import java.sql.SQLException;
+
+import java.io.IOException;
+
+import org.apache.derby.iapi.services.sanity.SanityManager;
+
+/**
+ * An <code>InputStream</code> that will use an locator to fetch the
+ * Blob value from the server.  
+ * <p>
+ * Closing a <code>ByteArrayInputStream</code> has no effect. The methods in
+ * this class can be called after the stream has been closed without
+ * generating an <code>IOException</code>.
+ * <p>
+ * This <code>InputStream</code> implementation is pretty basic.  No
+ * buffering of data is done.  Hence, for efficieny #read(byte[])
+ * should be used instead of #read().  Marks are not supported, but it
+ * should be pretty simple to extend the implementation to support
+ * this.  A more efficient skip implementation should also be
+ * straight-forward.
+ */
+public class BlobLocatorInputStream extends java.io.InputStream 
+{
+
+    /**
+     * Create an <code>InputStream</code> for reading the
+   * <code>Blob</code> value represented by the given locator based
+     * <code>Blob</code> object.
+     * @param connection connection to be used to read the
+     *        <code>Blob</code> value from the server
+     * @param blob <code>Blob</code> object that contains locator for
+   *        the <code>Blob</code> value on the server.
+     */
+    public BlobLocatorInputStream(Connection connection, Blob blob) 
+    {        
+        SanityManager.ASSERT(blob.isLocator());
+         this.connection = connection;
+        this.blob = blob;
+    }
+    
+    /**
+   * @see java.io.InputStream#read()
+     *
+     * This method fetches one byte at a time from the server. For more 
+     * efficient retrieval, use #read(byte[]).
+     */    
+  public int read() throws IOException
+    {
+        byte[] bytes = readBytes(1);
+        if (bytes.length == 0) { // EOF
+            return -1;
+      } else {
+            return bytes[0];
+        }
+    }
+    
+  /**
+     * @see java.io.InputStream#read(byte[], int, int)
+     */
+    public int read(byte[] b, int off, int len) throws IOException 
+    {
+      if (len == 0) return 0;
+        if ((off < 0) || (len < 0) || (off+len > b.length)) {
+            throw new IndexOutOfBoundsException();
+        }
+        
+      byte[] bytes = readBytes(len);
+        if (bytes.length == 0) { // EOF
+            return -1;
+        } else {
+            System.arraycopy(bytes, 0, b, off, bytes.length);
+          return bytes.length;
+        }
+    }
+
+    /**
+   * Read the next <code>len</code> bytes of the <code>Blob</code>
+     * value from the server.
+     * 
+     * @param len number of bytes to read
+     * @throws java.io.IOException Wrapped SqlException if reading
+   *         from server fails.
+     * @return <code>byte[]</code> containing the read bytes
+     */
+    private byte[] readBytes(int len) throws IOException
+    {
+      try {
+            int actualLength 
+                = (int )Math.min(len, blob.sqlLength() - currentPos + 1);
+            byte[] result = connection.locatorProcedureCall()
+                .blobGetBytes(blob.getLocator(), currentPos, actualLength);
+          currentPos += result.length;
+            return result;       
+        } catch (SqlException ex) {
+            IOException ioEx = new IOException();
+            ioEx.initCause(ex);
+          throw ioEx;
+        }
+    }
+    
+    
+  /**
+     * Connection used to read Blob from server.
+     */
+    private Connection connection;
+    
+  /**
+     * The Blob to be accessed.
+     */
+    private Blob blob;
+
+  /**
+     * Current position in the underlying Blob.
+     * Blobs are indexed from 1
+     */
+    private long currentPos = 1;
+}

Propchange: db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorInputStream.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorOutputStream.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorOutputStream.java?view=auto&rev=533268
==============================================================================
--- db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorOutputStream.java (added)
+++ db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorOutputStream.java Fri Apr 27 17:08:14 2007
@@ -0,0 +1,145 @@
+/*
+ 
+   Derby - Class org.apache.derby.client.am.BlobLocatorOutputStream
+ 
+   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.client.am;
+import java.io.IOException;
+
+/**
+ * An <code>OutputStream</code> that will use an locator to write
+ * bytes to the Blob value on the server.
+ * <p>
+ * Closing a <code>ByteArrayInputStream</code> has no effect. The methods in
+ * this class can be called after the stream has been closed without
+ * generating an <code>IOException</code>.
+ * <p>
+ * This <code>OutputStream</code> implementation is pretty basic.  No
+ * buffering of data is done.  Hence, for efficieny #write(byte[])
+ * should be used instead of #write(int).  
+ */
+public class BlobLocatorOutputStream extends java.io.OutputStream {
+    
+    /**
+     * Create an <code>OutputStream</code> for writing to the
+     * <code>Blob</code> value represented by the given locator based
+     * <code>Blob</code> object.
+     * @param connection connection to be used to write to the
+     *        <code>Blob</code> value on the server
+     * @param blob <code>Blob</code> object that contains locator for
+     *        the <code>Blob</code> value on the server.
+     * @param pos the position in the <code>BLOB</code> value at which
+     *        to start writing; the first position is 1
+     * @throws org.apache.derby.client.am.SqlException 
+     */
+    public BlobLocatorOutputStream(Connection connection, Blob blob, long pos)
+        throws SqlException
+    {
+        if (pos-1 > blob.sqlLength()) {
+            throw new IndexOutOfBoundsException();
+       }
+        
+        this.connection = connection;
+        this.blob = blob;
+        this.currentPos = pos;
+    }
+
+    /**
+     * @see java.io.OutputStream#write(int)
+     *
+     * This method writes one byte at a time to the server. For more 
+     * efficient writing, use #write(byte[]).
+     */
+    public void write(int b) throws IOException            
+    {
+        byte[] ba = {(byte )b};
+        writeBytes(ba);
+    }
+
+    /**
+     * @see java.io.OutputStream#write(byte[])
+     */
+    public void write(byte[] b) throws IOException 
+    {
+        writeBytes(b);
+    }
+    
+    
+    
+    /**
+     * @see java.io.OutputStream#write(byte[], int, int)
+     */
+    public void write(byte[] b, int off, int len) throws IOException 
+   {
+         if (len == 0) return;
+         if ((off < 0) || (off > b.length) || (len < 0) || 
+                 (off+len > b.length) || (off+len < 0)) {
+             throw new IndexOutOfBoundsException();
+         } 
+         
+         byte[] ba = b;
+         if ((off > 0) || (len < b.length)) { // Copy the part we will use
+             ba = new byte[len];
+             System.arraycopy(b, off, ba, 0, len);
+        }
+         writeBytes(ba);
+    }
+ 
+    /**
+     * Write the <code>byte[]</code> to the <code>Blob</code> value on
+     * the server; starting from the current position of this stream.
+     * 
+     * @param b The byte array containing the bytes to be written
+     * @throws java.io.IOException Wrapped SqlException if writing
+     *         to server fails.
+    */
+    private void writeBytes(byte[] b) throws IOException
+    {
+        try {         
+            connection.locatorProcedureCall()
+                .blobSetBytes(blob.locator_, currentPos, b.length, b);
+            currentPos += b.length;
+            if (currentPos-1 > blob.sqlLength()) {
+                // Wrote past the old end of the Blob value, update length
+                blob.setSqlLength(currentPos - 1);
+            }
+       } catch (SqlException ex) {
+            IOException ioEx= new IOException();
+            ioEx.initCause(ex);
+            throw ioEx;
+        }
+    }
+    
+    /**
+     * Connection used to read Blob from server.
+     */
+    private Connection connection;
+   
+    /**
+     * The Blob to be accessed.
+     */
+    private Blob blob;
+
+    /**
+     * Current position in the underlying Blob.
+     * Blobs are indexed from 1
+     */
+    private long currentPos;
+ 
+}

Propchange: db/derby/code/trunk/java/client/org/apache/derby/client/am/BlobLocatorOutputStream.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/client/org/apache/derby/client/am/Cursor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/client/org/apache/derby/client/am/Cursor.java?view=diff&rev=533268&r1=533267&r2=533268
==============================================================================
--- db/derby/code/trunk/java/client/org/apache/derby/client/am/Cursor.java (original)
+++ db/derby/code/trunk/java/client/org/apache/derby/client/am/Cursor.java Fri Apr 27 17:08:14 2007
@@ -365,7 +365,7 @@
     }
 
     // Build a Java int from a 4-byte signed binary representation.
-    private final int get_INTEGER(int column) {
+    protected final int get_INTEGER(int column) {
         return org.apache.derby.client.am.SignedBinary.getInt(dataBuffer_,
                 columnDataPosition_[column - 1]);
     }

Modified: db/derby/code/trunk/java/client/org/apache/derby/client/am/Lob.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/client/org/apache/derby/client/am/Lob.java?view=diff&rev=533268&r1=533267&r2=533268
==============================================================================
--- db/derby/code/trunk/java/client/org/apache/derby/client/am/Lob.java (original)
+++ db/derby/code/trunk/java/client/org/apache/derby/client/am/Lob.java Fri Apr 27 17:08:14 2007
@@ -40,12 +40,14 @@
     public static final int CHARACTER_STREAM = 16;
     public static final int BINARY_STREAM = 32;
     public static final int BINARY_STRING = 64;
+    public static final int LOCATOR = 128;
 
     //---------------------navigational members-----------------------------------
     protected Agent agent_;
 
     //-----------------------------state------------------------------------------
     protected int dataType_ = 0;      // data type(s) the LOB instance currently contains
+    protected int locator_ = -1;   // locator id for this LOB, -1 if not locator
 
     private long sqlLength_;// length of the LOB value, as defined by the server
     private boolean lengthObtained_;
@@ -72,18 +74,25 @@
     // ---------------------------jdbc 2------------------------------------------
 
     /**
-     * Return the length of the Lob value represented by this Lob object.
-     * If length is not already known, Lob will first be materialized.
-     * NOTE: The caller needs to deal with synchronization.
+     * Return the length of the Lob value represented by this Lob
+     * object.  If length is not already known, and Lob is locator
+     * based, length will be retrieved from the server.  If not,
+     * locator based, Lob will first be materialized.  NOTE: The
+     * caller needs to deal with synchronization.
      *
      * @throws SqlException on execution errors while materializing the stream, 
-     *         or if Layer B streaming is used and length not already obtained.
+     *         or if Layer B streaming is used and length not yet obtained.
      * @return length of Lob value
      */
     long sqlLength() throws SqlException 
     {
         if (lengthObtained_) return sqlLength_;
         
+        if (isLocator()) {
+            sqlLength_ = getLocatorLength();
+            lengthObtained_ = true;
+        }
+        
         if (willBeLayerBStreamed()) {
             throw new SqlException(agent_.logWriter_,
                                    LOB_OBJECT_LENGTH_UNKNOWN_YET);
@@ -106,6 +115,21 @@
         lengthObtained_ = true;
     }
 
+    /**
+     * Get the length of locator based Lob from the server.  This is a
+     * dummy implementation that is supposed to be overridden by
+     * subclasses.  A stored procedure call will be made to get the
+     * length from the server.
+     * 
+     * @throws org.apache.derby.client.am.SqlException 
+     * @return length of Lob
+     */
+    long getLocatorLength() throws SqlException
+    {
+        return -1;
+    }
+
+
     //-----------------------event callback methods-------------------------------
 
     public void listenToUnitOfWork() {
@@ -246,6 +270,23 @@
     public boolean willBeLayerBStreamed() {
         return willBeLayerBStreamed_;
     }
+
+    /**
+     * Check whether this Lob is based on a locator
+     * @return true if Lob is based on locator, false otherwise
+     */
+    public boolean isLocator() {
+        return ((dataType_ & LOCATOR) == LOCATOR);
+    }
+
+    /**
+     * Get locator for this Lob
+     * @return locator for this Lob, -1 is Lob is not based on locator
+     */
+    public int getLocator() {
+        return locator_;
+    }
+
 
     /**
      * Checks the <code>pos</code> and <code>length</code>.

Modified: db/derby/code/trunk/java/client/org/apache/derby/client/net/NetCursor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/client/org/apache/derby/client/net/NetCursor.java?view=diff&rev=533268&r1=533267&r2=533268
==============================================================================
--- db/derby/code/trunk/java/client/org/apache/derby/client/net/NetCursor.java (original)
+++ db/derby/code/trunk/java/client/org/apache/derby/client/net/NetCursor.java Fri Apr 27 17:08:14 2007
@@ -635,11 +635,13 @@
     // Calculates the column index for Lob objects constructed from EXTDTA data.
     // Describe information isn't sufficient because we have to check
     // for trivial values (nulls or zero-length) and exclude them.
+    // Need also to check whether locator was returned since in that case
+    // there will be no EXTDTA data for the LOB column.
     void calculateLobColumnPositionsForRow() {
         int currentPosition = 0;
 
         for (int i = 0; i < columns_; i++) {
-            if (isNonTrivialDataLob(i))
+            if ((isNonTrivialDataLob(i)) && (locator(i + 1) == -1))
             // key = column position, data = index to corresponding data in extdtaData_
             // ASSERT: the server always returns the EXTDTA objects in ascending order
             {
@@ -1045,8 +1047,32 @@
 
         return data;
     }
+    
+    /**
+     * Get locator for LOB of the designated column
+     * @param column column number, starts at 1
+     * @return locator value, -1 if LOB value was sent instead of locator
+     */
+    private int locator(int column)
+    {
+        int locator = get_INTEGER(column);
+        // If Lob value was sent instead of locator, highest bit will be set
+        if ((locator & 0x8000) == 0x8000) { 
+            return -1;
+        } else {
+            return locator;
+        }
+    }
 
-    public Blob getBlobColumn_(int column, Agent agent) throws SqlException {
+    public Blob getBlobColumn_(int column, Agent agent) throws SqlException 
+    {
+        // Check for locator
+        int locator = locator(column);
+        if (locator > 0) { // Create locator-based LOB object
+            return new Blob(agent, locator);
+        }
+        
+        // The Blob value has been sent instead of locator 
         int index = column - 1;
         int dataOffset;
         byte[] data;

Modified: db/derby/code/trunk/java/client/org/apache/derby/client/net/NetStatementRequest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/client/org/apache/derby/client/net/NetStatementRequest.java?view=diff&rev=533268&r1=533267&r2=533268
==============================================================================
--- db/derby/code/trunk/java/client/org/apache/derby/client/net/NetStatementRequest.java (original)
+++ db/derby/code/trunk/java/client/org/apache/derby/client/net/NetStatementRequest.java Fri Apr 27 17:08:14 2007
@@ -635,8 +635,11 @@
                     lidAndLengths[i][0] = DRDAConstants.DRDA_TYPE_NLOBLOC;
                     lidAndLengths[i][1] = 4;
                     break;
+ 
                 case java.sql.Types.CLOB:
-                    lidAndLengths[i][0] = DRDAConstants.DRDA_TYPE_NCLOBLOC;
+                    // Locators for Clob has not yet been implemented
+                    // lidAndLengths[i][0] = DRDAConstants.DRDA_TYPE_NCLOBLOC;
+                    lidAndLengths[i][0] = DRDAConstants.DRDA_TYPE_NLOBCMIXED;
                     lidAndLengths[i][1] = 4;
                     break;
             }