You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by sa...@apache.org on 2017/09/25 04:33:15 UTC

[2/6] incubator-trafodion git commit: [TRAFODION-2740]JDBC extract LOB context in chunks

[TRAFODION-2740]JDBC extract LOB context in chunks


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/commit/bd792ff7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/bd792ff7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/bd792ff7

Branch: refs/heads/release2.2
Commit: bd792ff7ef0a81c6b0106ce1d2aaeb3ff34d9701
Parents: 808c874
Author: Weiqing Xu <we...@esgyn.cn>
Authored: Wed Sep 20 05:57:26 2017 +0000
Committer: Weiqing Xu <we...@esgyn.cn>
Committed: Wed Sep 20 05:57:26 2017 +0000

----------------------------------------------------------------------
 .../trafodion/jdbc/t4/ExtractLobMessage.java    | 13 ++-
 .../org/trafodion/jdbc/t4/ExtractLobReply.java  | 23 ++++-
 .../java/org/trafodion/jdbc/t4/TrafT4Blob.java  | 11 ---
 .../java/org/trafodion/jdbc/t4/TrafT4Clob.java  | 13 ---
 .../java/org/trafodion/jdbc/t4/TrafT4Lob.java   | 37 +++++++-
 .../nsksrvr/Interface/marshalingsrvr_srvr.cpp   | 48 +++++++---
 .../nsksrvr/Interface/marshalingsrvr_srvr.h     |  6 +-
 .../src/odbc/nsksrvr/Interface/odbcs_srvr.cpp   | 11 ++-
 .../odbc/nsksrvr/Interface/odbcs_srvr_res.cpp   | 18 ++--
 .../src/odbc/nsksrvr/Interface/odbcs_srvr_res.h |  6 +-
 core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.cpp | 20 ++--
 core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.h   |  3 +-
 .../conn/odbc/src/odbc/nsksrvrcore/srvrcommon.h |  5 +-
 .../odbc/src/odbc/nsksrvrcore/srvrothers.cpp    | 99 +++++++-------------
 14 files changed, 174 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobMessage.java
----------------------------------------------------------------------
diff --git a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobMessage.java b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobMessage.java
index 6c54ff6..8a6c15c 100644
--- a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobMessage.java
+++ b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobMessage.java
@@ -27,9 +27,9 @@ class ExtractLobMessage {
 
 	static final short LOB_EXTRACT_LEN                 = 0;
 	static final short LOB_EXTRACT_BUFFER              = LOB_EXTRACT_LEN + 1;
-	static final short LOB_EXTRACT_BOTH_LEN_AND_BUFFER = LOB_EXTRACT_BUFFER + 1;
+	static final short LOB_CLOSE_CURSOR                = LOB_EXTRACT_BUFFER + 1;
 
-	static LogicalByteArray marshal(short extractType, String lobHandle, int lobHandleCharset, long lobLength, InterfaceConnection ic) throws SQLException{
+	static LogicalByteArray marshal(short extractType, String lobHandle, int lobHandleCharset, long extractLen, InterfaceConnection ic) throws SQLException{
 		int wlength = Header.sizeOf();
 		LogicalByteArray buf;
 
@@ -37,20 +37,23 @@ class ExtractLobMessage {
 			byte[] lobHandleBytes = ic.encodeString(lobHandle, InterfaceUtilities.SQLCHARSETCODE_UTF8);
 
 			wlength += TRANSPORT.size_int;
-			// wlength += TRANSPORT.size_long; // length of lobHandle
 
 			if (lobHandle.length() > 0) {
 				wlength += TRANSPORT.size_bytesWithCharset(lobHandleBytes);
 			}
 
-			if (lobLength > 0) {
+			if (extractLen > 0) {
 				wlength += TRANSPORT.size_long;
 			}
 
 			buf = new LogicalByteArray(wlength, Header.sizeOf(), ic.getByteSwap());
 
-			buf.insertInt(extractType);
+			buf.insertShort(extractType);
 			buf.insertStringWithCharset(lobHandleBytes, lobHandleCharset);
+			
+			if (extractType == LOB_EXTRACT_BUFFER) {
+				buf.insertLong(extractLen);
+			}
 			return buf;
 		} catch (Exception e) {
 			throw TrafT4Messages.createSQLException(ic.t4props_, ic.getLocale(), "unsupported_encoding", "UTF-8");

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobReply.java
----------------------------------------------------------------------
diff --git a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobReply.java b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobReply.java
index ca2c7f2..c54d854 100644
--- a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobReply.java
+++ b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/ExtractLobReply.java
@@ -30,9 +30,11 @@ class ExtractLobReply {
 	odbc_SQLSvc_ExtractLob_exc_ m_p1;
 	String proxySyntax = "";
 
-	int lobDataLen = 0;
+	long lobLength = 0;
+	long extractLen = 0;
+	byte[] extractData = null;
 
-	byte[] lobDataValue = null;
+	public short extractAPIType = 0;
 
 	ExtractLobReply(LogicalByteArray buf, InterfaceConnection ic) throws SQLException {
 		buf.setLocation(Header.sizeOf());
@@ -42,9 +44,20 @@ class ExtractLobReply {
 		m_p1.extractFromByteArray(buf, ic);
 
 		if (m_p1.exception_nr == TRANSPORT.CEE_SUCCESS) {
-			lobDataLen = (int) buf.extractInt();
-			if (lobDataLen > 0) {
-				lobDataValue = buf.extractByteArray(lobDataLen);
+
+			extractAPIType = buf.extractShort();
+			switch (extractAPIType) {
+			case 0:
+				lobLength = buf.extractLong();
+				break;
+			case 1:
+				extractLen = buf.extractLong();
+				extractData = buf.extractByteArray(extractLen);
+				break;
+			case 2:
+				break;
+			default:
+				break;
 			}
 		}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Blob.java
----------------------------------------------------------------------
diff --git a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Blob.java b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Blob.java
index 499dd82..fb76ea5 100644
--- a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Blob.java
+++ b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Blob.java
@@ -35,13 +35,11 @@ public class TrafT4Blob extends TrafT4Lob implements Blob
 		super(connection, lobHandle, data, Types.BLOB);
 	}
 
-	@Override
 	public InputStream getBinaryStream() throws SQLException {
 		testAvailability();
 		return new ByteArrayInputStream((byte[]) data_);
 	}
 
-	@Override
 	public InputStream getBinaryStream(long pos, long length) throws SQLException {
 		testAvailability();
 		return new ByteArrayInputStream((byte[]) data_);
@@ -53,7 +51,6 @@ public class TrafT4Blob extends TrafT4Lob implements Blob
 	 * bytes starting at position pos.
 	 *  */
 
-	@Override
 	public byte[] getBytes(long pos, int length) throws SQLException {
 		testAvailability();
 
@@ -75,14 +72,12 @@ public class TrafT4Blob extends TrafT4Lob implements Blob
 		return buf;
 	}
 
-	@Override
 	public int setBytes(long pos, byte[] bytes) throws SQLException {
 		testAvailability();
 
 		return setBytes(pos, bytes, 0, bytes.length);
 	}
 
-	@Override
 	public int setBytes(long pos, byte[] bytes, int offset, int len) throws SQLException {
 		testAvailability();
 		OutputStream out = setBinaryStream(pos);
@@ -102,24 +97,20 @@ public class TrafT4Blob extends TrafT4Lob implements Blob
 		return len;
 	}
 
-	@Override
 	public long position(Blob pattern, long start) throws SQLException {
 		return position(pattern.getBytes(0, (int) pattern.length()), start);
 	}
 
-	@Override
 	public long position(byte[] pattern, long start) throws SQLException {
 		TrafT4Messages.throwUnsupportedFeatureException(connection_.props_, connection_.getLocale(), "position()");
 		return 0;
 	}
 
-	@Override
 	public OutputStream setBinaryStream(long pos) throws SQLException {
 		testAvailability();
 		return setOutputStream(pos);
 	}
 
-	@Override
 	public void truncate(long len) throws SQLException {
 		testAvailability();
 
@@ -145,13 +136,11 @@ public class TrafT4Blob extends TrafT4Lob implements Blob
 	}
 
 
-	@Override
 	public long length() throws SQLException {
 		testAvailability();
 		return data_ == null ? 0 : ((byte[]) data_).length;
 	}
 
-	@Override
 	public void free() throws SQLException {
 		data_ = null;
 		isFreed_ = true;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Clob.java
----------------------------------------------------------------------
diff --git a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Clob.java b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Clob.java
index f7280fe..5877089 100644
--- a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Clob.java
+++ b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Clob.java
@@ -42,13 +42,11 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 			this.data_ = "";
 	}
 
-	@Override
 	public void free() throws SQLException {
 		data_ = null;
 		isFreed_ = true;
 	}
 
-	@Override
 	public InputStream getAsciiStream() throws SQLException {
 		testAvailability();
 		if (data_ != null) {
@@ -58,7 +56,6 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 		return null;
 	}
 
-	@Override
 	public Reader getCharacterStream() throws SQLException {
 		testAvailability();
 
@@ -69,14 +66,12 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 		return null;
 	}
 
-	@Override
 	public Reader getCharacterStream(long pos, long length) throws SQLException {
 		testAvailability();
 
 		return new StringReader(getSubString(pos, (int) length));
 	}
 
-	@Override
 	public String getSubString(long pos, int length) throws SQLException {
 		testAvailability();
 
@@ -94,7 +89,6 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 		return ((String) data_).substring(beginIndex, endIndex);
 	}
 
-	@Override
 	public long length() throws SQLException {
 		testAvailability();
 		if (data_ != null) {
@@ -104,7 +98,6 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 		return 0;
 	}
 
-	@Override
 	public long position(String searchstr, long start) throws SQLException {
 		testAvailability();
 		//start--;
@@ -123,12 +116,10 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 		return pos;
 	}
 
-	@Override
 	public long position(Clob searchstr, long start) throws SQLException {
 		return position(searchstr.getSubString(1L, (int) searchstr.length()), start);
 	}
 
-	@Override
 	public OutputStream setAsciiStream(long pos) throws SQLException {
 		testAvailability();
 		if (pos < 1) {
@@ -139,14 +130,12 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 		return setOutputStream(pos);
 	}
 
-	@Override
 	public Writer setCharacterStream(long pos) throws SQLException {
 		testAvailability();
 		TrafT4Writer writer = new TrafT4Writer(this, pos);
 		return writer;
 	}
 
-	@Override
 	public int setString(long pos, String str) throws SQLException {
 		testAvailability();
 		int startIndex = (int) pos - 1;
@@ -168,7 +157,6 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 		return len;
 	}
 
-	@Override
 	public int setString(long pos, String str, int offset, int len) throws SQLException {
 		testAvailability();
 		int start = (int) pos - 1;
@@ -183,7 +171,6 @@ public class TrafT4Clob extends TrafT4Lob implements Clob {
 		return len;
 	}
 
-	@Override
 	public void truncate(long len) throws SQLException {
 		testAvailability();
 		if (len > ((String) data_).length()) {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Lob.java
----------------------------------------------------------------------
diff --git a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Lob.java b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Lob.java
index 054da20..f18377b 100644
--- a/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Lob.java
+++ b/core/conn/jdbcT4/src/main/java/org/trafodion/jdbc/t4/TrafT4Lob.java
@@ -34,6 +34,7 @@ public abstract class TrafT4Lob {
 	protected TrafT4Connection connection_ = null;
 	protected boolean isFreed_ = true;
 	protected int lobType = Types.BLOB;
+	protected long length = 0;
 
 	protected Object data_ = null;
 
@@ -91,23 +92,49 @@ public abstract class TrafT4Lob {
 		}
 
 		T4Connection t4connection = this.connection_.getServerHandle().getT4Connection();
-		LogicalByteArray wbuffer = ExtractLobMessage.marshal(ExtractLobMessage.LOB_EXTRACT_BUFFER, lobHandle_, 1, 0,
+		LogicalByteArray wbuffer = ExtractLobMessage.marshal(ExtractLobMessage.LOB_EXTRACT_LEN, lobHandle_, 1, 0,
 				connection_.ic_);
 
 		LogicalByteArray rbuffer = t4connection.getReadBuffer(TRANSPORT.SRVR_API_EXTRACTLOB, wbuffer);
 		ExtractLobReply reply = new ExtractLobReply(rbuffer, connection_.ic_);
-		data_ = reply.lobDataValue;
+		length = reply.lobLength;
+
+		byte[] fetchData_ = new byte[(int) length];
+
+		try {
+			int pos = 0;
+			int chunkSize = 100 * 1024 * 1024;
+			while (pos < length) {
+				int remainSize = (int) (length - pos);
+				int fecthSize = remainSize < chunkSize ? remainSize : chunkSize;
+				wbuffer =  ExtractLobMessage.marshal(ExtractLobMessage.LOB_EXTRACT_BUFFER, lobHandle_, 1, fecthSize, connection_.ic_);
+				rbuffer = t4connection.getReadBuffer(TRANSPORT.SRVR_API_EXTRACTLOB, wbuffer);
+				reply = new ExtractLobReply(rbuffer, connection_.ic_);
+				System.arraycopy(reply.extractData, 0, fetchData_, pos, (int) reply.extractLen);
+				pos += reply.extractLen;
+			}
+
+		}
+		catch(SQLException se) {
+			throw se;
+		}
+		finally {
+			// close the LOB cursor
+			wbuffer = ExtractLobMessage.marshal(ExtractLobMessage.LOB_CLOSE_CURSOR, lobHandle_, 1, 0, connection_.ic_);
+			rbuffer = t4connection.getReadBuffer(TRANSPORT.SRVR_API_EXTRACTLOB, wbuffer);
+			reply = new ExtractLobReply(rbuffer, connection_.ic_);
+		}
 		switch (lobType) {
 		case Types.BLOB:
-			data_ = reply.lobDataValue;
+			data_ = fetchData_;
 			break;
 		case Types.CLOB:
 			try {
-			    if (reply.lobDataLen == 0) {
+			    if (length == 0) {
 			        data_ = "";
 			    }
 			    else {
-			        data_ = new String(reply.lobDataValue, "UTF-8");
+			        data_ = new String(fetchData_, "UTF-8");
 			    }
 			} catch (UnsupportedEncodingException e) {
 				throw TrafT4Messages.createSQLException(this.connection_.ic_.t4props_, this.connection_.ic_.getLocale(),

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.cpp
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.cpp b/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.cpp
index 63dfbb7..30b70ba 100644
--- a/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.cpp
+++ b/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.cpp
@@ -2307,8 +2307,10 @@ odbc_SQLsrvr_ExtractLob_param_res_(
       , char* &buffer
       , UInt32& message_length
       , const struct odbc_SQLsrvr_ExtractLob_exc_ *exception_
-      , IDL_long_long lobDataLen
-      , BYTE * lobDataValue
+      , IDL_short extractLobAPI
+      , IDL_long_long lobLength
+      , IDL_long_long extractLen
+      , BYTE * extractData
 )
 {
     CEE_status sts = CEE_SUCCESS;
@@ -2352,13 +2354,20 @@ odbc_SQLsrvr_ExtractLob_param_res_(
             break;
     }
 
-    // length of IDL_long  LOB len
-    wlength += sizeof(IDL_long);
-    if (lobDataValue != NULL)
-    {
-        wlength += lobDataLen;
+    wlength += sizeof(IDL_short);
+    switch (extractLobAPI) {
+    case 0:
+        wlength += sizeof(IDL_long_long);
+        break;
+    case 1:
+        wlength += sizeof(IDL_long_long);
+        wlength += extractLen;
+        break;
+    case 2:
+        break;
+    default:
+        break;
     }
-    wlength += lobDataLen;
 
     // update the length of message
     message_length = wlength;
@@ -2394,10 +2403,25 @@ odbc_SQLsrvr_ExtractLob_param_res_(
         default:
             break;
     }
-    IDL_long_copy((IDL_long *)&lobDataLen, curptr);
-    if (lobDataValue != NULL)
-    {
-        IDL_byteArray_copy(lobDataValue, lobDataLen, curptr);
+
+    //IDL_long_copy((IDL_long *)&extractLobAPI, curptr);
+    IDL_short_copy((IDL_short *)&extractLobAPI, curptr);
+
+    switch (extractLobAPI) {
+    case 0:
+        IDL_long_long_copy((IDL_long_long *)&lobLength, curptr);
+        break;
+    case 1:
+        IDL_long_long_copy((IDL_long_long *)&extractLen, curptr);
+        if (extractLen != 0)
+        {
+            IDL_byteArray_copy(extractData, extractLen, curptr);
+        }
+        break;
+    case 2:
+        break;
+    default:
+        break;
     }
 
     return sts;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.h
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.h b/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.h
index 8d21291..af3e6ae 100644
--- a/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.h
+++ b/core/conn/odbc/src/odbc/nsksrvr/Interface/marshalingsrvr_srvr.h
@@ -388,8 +388,10 @@ odbc_SQLsrvr_ExtractLob_param_res_(
                 , char *&buffer
                 , UInt32 & message_length
                 , const struct odbc_SQLsrvr_ExtractLob_exc_ *exception_
-                , IDL_long_long lobDataLen
-                , BYTE * lobDataValue
+                , IDL_short extractLobAPI
+                , IDL_long_long lobLength
+                , IDL_long_long extractLen
+                , BYTE * extractData
 );
 
 CEE_status

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr.cpp
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr.cpp b/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr.cpp
index 5fba9e1..a6f6639 100644
--- a/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr.cpp
+++ b/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr.cpp
@@ -1526,14 +1526,15 @@ EXTRACTLOB_IOMessage(
     IDL_char     *curptr;
     IDL_long inputPosition = 0;
 
-    IDL_long extractLobAPI = 0;
+    IDL_short extractLobAPI = 0;
+    IDL_long extractLen = 0;
     IDL_long lobHandleLen = 0;
     IDL_string lobHandle = NULL;
     IDL_long lobHandleCharset = 0;
 
     curptr = pnode->r_buffer();
 
-    extractLobAPI = *(IDL_long *)(curptr + inputPosition);
+    extractLobAPI = *(IDL_short *)(curptr + inputPosition);
     inputPosition += sizeof(extractLobAPI);
 
     lobHandleLen = *(IDL_long*)(curptr + inputPosition);
@@ -1547,11 +1548,15 @@ EXTRACTLOB_IOMessage(
         inputPosition += sizeof(lobHandleCharset);
     }
 
+    extractLen = *(IDL_long *)(curptr + inputPosition);
+    inputPosition += sizeof(extractLen);
+
     odbc_SQLSrvr_ExtractLob_ame_(
             objtag_,
             call_id_,
             extractLobAPI,
-            lobHandle
+            lobHandle,
+            extractLen
             );
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.cpp
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.cpp b/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.cpp
index 48602b7..5a34c2c 100644
--- a/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.cpp
+++ b/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.cpp
@@ -647,8 +647,10 @@ odbc_SQLSrvr_ExtractLob_ts_res_(
     /* In   */ CEE_tag_def    objtag_
   , /* In   */ const CEE_handle_def *call_id_
   , /* In   */ const struct odbc_SQLsrvr_ExtractLob_exc_ *exception_
-  , /* In   */ IDL_long_long  lobDataLen
-  , /* In   */ BYTE       *lobDataValue
+  , /* In   */ IDL_short extractLobAPI
+  , /* In   */ IDL_long_long  lobLength
+  , /* In   */ IDL_long_long  extractLen
+  , /* In   */ BYTE   *  extractData
   )
 {
     CInterface* pnode = (CInterface *)objtag_;
@@ -666,17 +668,19 @@ odbc_SQLSrvr_ExtractLob_ts_res_(
             , buffer
             , message_length
             , exception_
-            , lobDataLen
-            , lobDataValue
+            , extractLobAPI
+            , lobLength
+            , extractLen
+            , extractData
             );
 
     if (sts == CEE_SUCCESS)
         sts = pnode->send_response(buffer, message_length, call_id_);
 
-    if (lobDataValue != NULL)
+    if (extractData != NULL)
     {
-        delete [] lobDataValue;
-        lobDataValue = NULL;
+        delete [] extractData;
+        extractData = NULL;
     }
     return sts;
 }

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.h
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.h b/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.h
index fba2e39..7e5f51d 100644
--- a/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.h
+++ b/core/conn/odbc/src/odbc/nsksrvr/Interface/odbcs_srvr_res.h
@@ -346,8 +346,10 @@ odbc_SQLSrvr_ExtractLob_ts_res_(
     /* In    */ CEE_tag_def objtag_
   , /* In    */ const CEE_handle_def *call_id_
   , /* In    */ const struct odbc_SQLsrvr_ExtractLob_exc_ *exception_
-  , /* In    */ IDL_long_long lobDataLen
-  , /* In    */ BYTE * lobDataValue
+  , /* In   */ IDL_short  extractLobAPI
+  , /* In   */ IDL_long_long  lobLength
+  , /* In   */ IDL_long_long  extractLen
+  , /* In   */ BYTE   *  extractData
 );
 
 void

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.cpp
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.cpp b/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.cpp
index 39f363e..fb623e2 100644
--- a/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.cpp
+++ b/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.cpp
@@ -8534,12 +8534,13 @@ void
 odbc_SQLSrvr_ExtractLob_ame_(
     /* In   */ CEE_tag_def objtag_
   , /* In   */ const CEE_handle_def *call_id_
-  , /* In   */ IDL_long    extractLobAPI
-  , /* In   */ IDL_string  lobHandle)
+  , /* In   */ IDL_short   extractLobAPI
+  , /* In   */ IDL_string  lobHandle
+  , /* In   */ IDL_long_long    extractLen)
 {
     ERROR_DESC_LIST_def sqlWarning = {0, 0};
-    IDL_long_long lobDataLen = 0;
-    BYTE * lobDataValue = NULL;
+    IDL_long_long lobLength = 0;
+    BYTE * extractData = NULL;
 
     odbc_SQLsrvr_ExtractLob_exc_ exception_ = {0, 0};
 
@@ -8548,14 +8549,17 @@ odbc_SQLSrvr_ExtractLob_ame_(
                                  &exception_,
                                  extractLobAPI,
                                  lobHandle,
-                                 lobDataLen,
-                                 lobDataValue);
+                                 lobLength,
+                                 extractLen,
+                                 extractData);
 
     odbc_SQLSrvr_ExtractLob_ts_res_(objtag_,
                                     call_id_,
                                     &exception_,
-                                    lobDataLen,
-                                    lobDataValue);
+                                    extractLobAPI,
+                                    lobLength,
+                                    extractLen,
+                                    extractData);
 }
 
 void

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.h
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.h b/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.h
index 66556e8..d429fc8 100644
--- a/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.h
+++ b/core/conn/odbc/src/odbc/nsksrvr/SrvrConnect.h
@@ -336,8 +336,9 @@ extern "C" void
 odbc_SQLSrvr_ExtractLob_ame_(
     /* In   */ CEE_tag_def objtag_
   , /* In   */ const CEE_handle_def *call_id_
-  , /* In   */ IDL_long    extractLobAPI
+  , /* In   */ IDL_short    extractLobAPI
   , /* In   */ IDL_string  lobHandle
+  , /* In   */ IDL_long_long   extractLen
   );
 
 void

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvrcore/srvrcommon.h
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvrcore/srvrcommon.h b/core/conn/odbc/src/odbc/nsksrvrcore/srvrcommon.h
index 6ed6692..ec5315c 100644
--- a/core/conn/odbc/src/odbc/nsksrvrcore/srvrcommon.h
+++ b/core/conn/odbc/src/odbc/nsksrvrcore/srvrcommon.h
@@ -387,8 +387,9 @@ odbc_SQLSrvr_ExtractLob_sme_(
   , /* In    */ odbc_SQLsrvr_ExtractLob_exc_ *exception_
   , /* In    */ IDL_long extractLobAPI
   , /* In    */ IDL_string lobHandle
-  , /* In    */ IDL_long_long &lobDataLen
-  , /* In    */ BYTE* &lobDataValue);
+  , /* In    */ IDL_long_long &lobLength
+  , /* Out   */ IDL_long_long &extractLen
+  , /* Out   */ BYTE *& extractData);
 
 extern "C" void
 odbc_SQLSrvr_UpdateLob_sme_(

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/bd792ff7/core/conn/odbc/src/odbc/nsksrvrcore/srvrothers.cpp
----------------------------------------------------------------------
diff --git a/core/conn/odbc/src/odbc/nsksrvrcore/srvrothers.cpp b/core/conn/odbc/src/odbc/nsksrvrcore/srvrothers.cpp
index ccbb689..af63ca3 100644
--- a/core/conn/odbc/src/odbc/nsksrvrcore/srvrothers.cpp
+++ b/core/conn/odbc/src/odbc/nsksrvrcore/srvrothers.cpp
@@ -6388,13 +6388,14 @@ odbc_SQLSrvr_ExtractLob_sme_(
     /* In    */ CEE_tag_def objtag_
   , /* In    */ const CEE_handle_def *call_id_
   , /* Out   */ odbc_SQLsrvr_ExtractLob_exc_ *exception_
-  , /* In    */ IDL_long extractLobAPI
+  , /* In    */ IDL_short extractLobAPI
   , /* In    */ IDL_string lobHandle
-  , /* Out   */ IDL_long_long &lobDataLen
-  , /* Out   */ BYTE *& lobDataValue
+  , /* In    */ IDL_long_long &lobLength
+  , /* Out   */ IDL_long_long &extractLen
+  , /* Out   */ BYTE *& extractData
   )
 {
-    char LobExtractQuery[1000];
+    char LobExtractQuery[1000] = {0};
     char RequestError[200] = {0};
     SRVR_STMT_HDL  *QryLobExtractSrvrStmt = NULL;
 
@@ -6411,8 +6412,34 @@ odbc_SQLSrvr_ExtractLob_sme_(
         exception_->exception_nr = odbc_SQLsrvr_ExtractLob_ParamError_exn_;
         exception_->u.ParamError.ParamDesc = SQLSVC_EXCEPTION_UNABLE_TO_ALLOCATE_SQL_STMT;
     }
+FILE * fp= NULL;
+        fp = fopen("/tmp/closeLOB", "a+");
+    switch (extractLobAPI) {
+    case 0:
+        fprintf(fp, "new LOB API\n");
+        extractData = NULL;
+        snprintf(LobExtractQuery, sizeof(LobExtractQuery), "EXTRACT LOBLENGTH(LOB'%s') LOCATION %Ld", lobHandle, (Int64)&lobLength);
+        break;
+    case 1:
+        extractData = new BYTE[extractLen + 1];
+        if (extractData == NULL)
+        {
+            exception_->exception_nr = odbc_SQLsrvr_ExtractLob_ParamError_exn_;
+            exception_->u.ParamError.ParamDesc = SQLSVC_EXCEPTION_BUFFER_ALLOC_FAILED;
+        }
 
-    snprintf(LobExtractQuery, sizeof(LobExtractQuery), "EXTRACT LOBLENGTH(LOB'%s') LOCATION %Ld", lobHandle, (Int64)&lobDataLen);
+        snprintf(LobExtractQuery, sizeof(LobExtractQuery), "EXTRACT LOBTOBUFFER(LOB'%s', LOCATION %Ld, SIZE %Ld)", lobHandle, (Int64)extractData, &extractLen);
+        break;
+    case 102:
+        extractLen = 0;
+        extractData = NULL;
+        snprintf(LobExtractQuery, sizeof(LobExtractQuery), "EXTRACT LOBTOBUFFER(LOB'%s', LOCATION %Ld, SIZE %Ld)", lobHandle, (Int64)extractData, &extractLen);
+        fprintf(fp, "close\n");
+        fclose(fp);
+        break;
+    default:
+        return ;
+    }
 
     try
     {
@@ -6446,70 +6473,13 @@ odbc_SQLSrvr_ExtractLob_sme_(
                 ODBCMX_SERVER,
                 srvrGlobal->srvrObjRef,
                 1,
+                    //"Exception in executing EXTRACT LOBTOBUFFER");
                 "Exception in executing EXTRACT LOBLENGTH");
 
         exception_->exception_nr = odbc_SQLsrvr_ExtractLob_ParamError_exn_;
         exception_->u.ParamError.ParamDesc = SQLSVC_EXCEPTION_EXECDIRECT_FAILED;
     }
 
-    lobDataValue = new BYTE[lobDataLen + 1];
-    if (lobDataValue == NULL)
-    {
-        exception_->exception_nr = odbc_SQLsrvr_ExtractLob_ParamError_exn_;
-        exception_->u.ParamError.ParamDesc = SQLSVC_EXCEPTION_BUFFER_ALLOC_FAILED;
-    }
-
-    memset(lobDataValue, 0, lobDataLen + 1);
-
-    memset(LobExtractQuery, 0, sizeof(LobExtractQuery));
-
-    snprintf(LobExtractQuery, sizeof(LobExtractQuery), "EXTRACT LOBTOBUFFER(LOB'%s', LOCATION %Ld, SIZE %Ld)", lobHandle, (Int64)lobDataValue, &lobDataLen);
-
-    if (exception_->exception_nr == 0)
-    {
-        try
-        {
-            short retcode = QryLobExtractSrvrStmt->ExecDirect(NULL, LobExtractQuery, EXTERNAL_STMT, TYPE_CALL, SQL_ASYNC_ENABLE_OFF, 0);
-            if (retcode == SQL_ERROR)
-            {
-                ERROR_DESC_def *p_buffer = QryLobExtractSrvrStmt->sqlError.errorList._buffer;
-                strncpy(RequestError, p_buffer->errorText, sizeof(RequestError) - 1);
-
-                SendEventMsg(MSG_SQL_ERROR,
-                        EVENTLOG_ERROR_TYPE,
-                        srvrGlobal->nskProcessInfo.processId,
-                         ODBCMX_SERVER,
-                         srvrGlobal->srvrObjRef,
-                         2,
-                         p_buffer->sqlcode,
-                         RequestError);
-
-                exception_->exception_nr = odbc_SQLsrvr_ExtractLob_ParamError_exn_;
-                exception_->u.SQLError.errorList._length = QryLobExtractSrvrStmt->sqlError.errorList._length;
-                exception_->u.SQLError.errorList._buffer = QryLobExtractSrvrStmt->sqlError.errorList._buffer;
-                exception_->u.ParamError.ParamDesc = SQLSVC_EXCEPTION_EXECUTE_FAILED;
-            }
-        }
-        catch (...)
-        {
-            SendEventMsg(MSG_PROGRAMMING_ERROR,
-                    EVENTLOG_ERROR_TYPE,
-                    srvrGlobal->nskProcessInfo.processId,
-                    ODBCMX_SERVER,
-                    srvrGlobal->srvrObjRef,
-                    1,
-                    "Exception in executing EXTRACT LOBTOBUFFER");
-
-            exception_->exception_nr = odbc_SQLsrvr_ExtractLob_ParamError_exn_;
-            exception_->u.ParamError.ParamDesc = SQLSVC_EXCEPTION_EXECDIRECT_FAILED;
-        }
-
-        if (exception_->exception_nr != 0) {
-            lobDataLen = 0;
-            delete [] lobDataValue;
-            lobDataValue = NULL;
-        }
-    }
 }
 
 extern "C" void
@@ -6591,6 +6561,9 @@ odbc_SQLSrvr_UpdateLob_sme_(
         exception_->u.ParamError.ParamDesc = SQLSVC_EXCEPTION_EXECUTE_FAILED;
     }
 
+    if (QryLobUpdateSrvrStmt != NULL) {
+        QryLobUpdateSrvrStmt->Close(SQL_DROP);
+    }
 }
 
 //========================================================================