You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2014/10/19 04:36:31 UTC

[1/2] git commit: PHOENIX-1242 ArrayoutofBoundException Phoenix mapping to exisiting Hbase (Alicia Ying Shu)

Repository: phoenix
Updated Branches:
  refs/heads/master a2fbf489d -> ed8c82a05


PHOENIX-1242 ArrayoutofBoundException Phoenix mapping to exisiting Hbase (Alicia Ying Shu)


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

Branch: refs/heads/master
Commit: ed8c82a05bf2887bd78f462afb6eaf834080a2f4
Parents: 6d6926d
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Oct 18 16:56:35 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Oct 18 19:41:20 2014 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/CoalesceFunctionIT.java     |   9 +-
 .../phoenix/end2end/DecodeFunctionIT.java       |  37 +-
 .../phoenix/end2end/MappingTableDataTypeIT.java |  98 ++++
 .../phoenix/expression/LiteralExpression.java   |  37 +-
 .../expression/function/DecodeFunction.java     |  12 +-
 .../org/apache/phoenix/schema/PDataType.java    | 489 ++++++++++---------
 .../phoenix/compile/QueryCompilerTest.java      |   2 +-
 .../schema/MappingTableDataTypeTest.java        | 119 -----
 .../apache/phoenix/schema/PDataTypeTest.java    |   8 +-
 .../org/apache/phoenix/util/ColumnInfoTest.java |  18 +-
 10 files changed, 414 insertions(+), 415 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
index 45fcb48..2a037fa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
@@ -23,6 +23,7 @@ import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -31,7 +32,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Properties;
 
-import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Assert;
 import org.junit.Test;
@@ -263,9 +264,9 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
 
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
-            Assert.fail("Should not cast -2 to UNSIGNED_INT");
-        } catch (IllegalDataException e) {
-
+            fail("Should not cast -2 to UNSIGNED_INT");
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
index 05e2504..6852f74 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -27,7 +28,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.schema.PDataType;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -82,11 +83,10 @@ public class DecodeFunctionIT extends BaseHBaseManagedTimeIT {
 
 		try {
 			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('zzxxuuyyzzxxuuyy', 'hex')");
-		} catch (IllegalDataException e) {
-			assertTrue(true);
-			return;
+	        fail();
+		} catch (SQLException e) {
+			assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
 		}
-		fail();
 	}
 
 	@Test
@@ -98,11 +98,10 @@ public class DecodeFunctionIT extends BaseHBaseManagedTimeIT {
 
 		try {
 			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('8', 'hex')");
-		} catch (IllegalDataException e) {
-			assertTrue(true);
-			return;
-		}
-		fail();
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
+        }
 	}
 
 	@Test
@@ -114,11 +113,10 @@ public class DecodeFunctionIT extends BaseHBaseManagedTimeIT {
 
 		try {
 			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('8', NULL)");
-		} catch (IllegalDataException e) {
-			assertTrue(true);
-			return;
-		}
-		fail();
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
+        }
 	}
 
 	@Test
@@ -130,10 +128,9 @@ public class DecodeFunctionIT extends BaseHBaseManagedTimeIT {
 
 		try {
 			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('8', 'someNonexistFormat')");
-		} catch (SQLException e) {
-			assertTrue(true);
-			return;
-		}
-		fail();
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.TYPE_MISMATCH.getErrorCode(), e.getErrorCode());
+        }
 	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
new file mode 100644
index 0000000..3c789e3
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
@@ -0,0 +1,98 @@
+/*
+ * 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.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(HBaseManagedTimeTest.class)
+public class MappingTableDataTypeIT extends BaseHBaseManagedTimeIT {
+    @Test
+    public void testMappingHbaseTableToPhoenixTable() throws Exception {
+        final TableName tableName = TableName.valueOf("MTEST");
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        PhoenixConnection conn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
+        
+        HBaseAdmin admin = conn.getQueryServices().getAdmin();
+        try {
+            // Create table then get the single region for our new table.
+            HTableDescriptor descriptor = new HTableDescriptor(tableName);
+            HColumnDescriptor columnDescriptor =  new HColumnDescriptor(Bytes.toBytes("cf"));
+            descriptor.addFamily(columnDescriptor);
+            admin.createTable(descriptor);
+            HTableInterface t = conn.getQueryServices().getTable(Bytes.toBytes("MTEST"));
+            insertData(tableName.getName(), admin, t);
+            t.close();
+            try {
+                testCreateTableMismatchedType();
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(),e.getErrorCode());
+            }
+        } finally {
+            admin.close();
+        }
+    }
+
+    private void insertData(final byte[] tableName, HBaseAdmin admin, HTableInterface t) throws IOException,
+            InterruptedException {
+        Put p = new Put(Bytes.toBytes("row"));
+        p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("value1"));
+        t.put(p);
+        t.flushCommits();
+        admin.flush(tableName);
+    }
+
+    /**
+     * Test create a table in Phoenix with mismatched data type UNSIGNED_LONG
+     */
+    private void testCreateTableMismatchedType() throws SQLException {
+        String ddl = "create table IF NOT EXISTS MTEST (" + " id varchar NOT NULL primary key,"
+                + " \"cf\".\"q1\" unsigned_long" + " ) ";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.createStatement().execute(ddl);
+        conn.commit();
+        String query = "select * from MTEST";
+        ResultSet rs = conn.createStatement().executeQuery(query);
+        rs.next();
+        rs.getLong(2);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
index d1ec046..e9daf2d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
@@ -24,10 +24,7 @@ import java.sql.SQLException;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableUtils;
-import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
-import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.PhoenixArray;
 import org.apache.phoenix.schema.SortOrder;
@@ -163,27 +160,23 @@ public class LiteralExpression extends BaseTerminalExpression {
             throw TypeMismatchException.newException(type, actualType, value.toString());
         }
         value = type.toObject(value, actualType);
-        try {
-            byte[] b = type.toBytes(value, sortOrder);
-            if (type == PDataType.VARCHAR || type == PDataType.CHAR) {
-                if (type == PDataType.CHAR && maxLength != null  && b.length < maxLength) {
-                    b = StringUtil.padChar(b, maxLength);
-                } else if (value != null) {
-                    maxLength = ((String)value).length();
-                }
-            } else if (type.isArrayType()) {
-                maxLength = ((PhoenixArray)value).getMaxLength();
+        byte[] b = type.toBytes(value, sortOrder);
+        if (type == PDataType.VARCHAR || type == PDataType.CHAR) {
+            if (type == PDataType.CHAR && maxLength != null  && b.length < maxLength) {
+                b = StringUtil.padChar(b, maxLength);
+            } else if (value != null) {
+                maxLength = ((String)value).length();
             }
-            if (b.length == 0) {
-                return getTypedNullLiteralExpression(type, determinism);
-            }
-            if (maxLength == null) {
-                maxLength = type == null || !type.isFixedWidth() ? null : type.getMaxLength(value);
-            }
-            return new LiteralExpression(value, type, b, maxLength, scale, sortOrder, determinism);
-        } catch (IllegalDataException e) {
-            throw new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA).setRootCause(e).build().buildException();
+        } else if (type.isArrayType()) {
+            maxLength = ((PhoenixArray)value).getMaxLength();
+        }
+        if (b.length == 0) {
+            return getTypedNullLiteralExpression(type, determinism);
+        }
+        if (maxLength == null) {
+            maxLength = type == null || !type.isFixedWidth() ? null : type.getMaxLength(value);
         }
+        return new LiteralExpression(value, type, b, maxLength, scale, sortOrder, determinism);
     }
 
     public LiteralExpression() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java
index f191eb4..2107ab9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DecodeFunction.java
@@ -19,7 +19,10 @@ package org.apache.phoenix.expression.function;
 
 import java.sql.SQLException;
 import java.util.List;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.parse.FunctionParseNode;
 import org.apache.phoenix.schema.IllegalDataException;
@@ -62,7 +65,8 @@ public class DecodeFunction extends ScalarFunction {
 		}
 
 		if (ptr.getLength() == 0) {
-			throw new IllegalDataException("Missing bytes encoding.");
+	        throw new IllegalDataException(new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA)
+	        .setMessage("Missing bytes encoding").build().buildException());
 		}
 
 		type = encodingExpression.getDataType();
@@ -89,9 +93,11 @@ public class DecodeFunction extends ScalarFunction {
 			try {
 				out[i / 2] = (byte) Integer.parseInt(hexStr.substring(i, i + 2), 16);
 			} catch (NumberFormatException ex) {
-				throw new IllegalDataException("Value " + hexStr.substring(i, i + 2) + " cannot be cast to hex number");
+				throw new IllegalDataException(new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA)
+		        .setMessage("Value " + hexStr.substring(i, i + 2) + " cannot be cast to hex number").build().buildException());
 			} catch (StringIndexOutOfBoundsException ex) {
-				throw new IllegalDataException("Invalid value length, cannot cast to hex number (" + hexStr + ")");
+				throw new IllegalDataException(new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA)
+                .setMessage("Invalid value length, cannot cast to hex number (" + hexStr + ")").build().buildException());
 			}
 		}
 		return out;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
index c1d1936..4c1701c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
@@ -33,6 +33,8 @@ import java.util.Random;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.exception.ValueTypeIncompatibleException;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
@@ -85,7 +87,7 @@ public enum PDataType {
         @Override
         public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (!actualType.isCoercibleTo(this)) {
-                throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
+                throwConstraintViolationException(actualType, this);
             }
             if (length == 0) {
                 return null;
@@ -224,11 +226,11 @@ public enum PDataType {
         @Override
         public byte[] toBytes(Object object) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             byte[] b = VARCHAR.toBytes(object);
             if (b.length != ((String) object).length()) {
-                throw new IllegalDataException("CHAR types may only contain single byte characters (" + object + ")");
+                throw newIllegalDataException("CHAR types may only contain single byte characters (" + object + ")");
             }
             return b;
         }
@@ -236,11 +238,11 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             int len = VARCHAR.toBytes(object, bytes, offset);
             if (len != ((String) object).length()) {
-                throw new IllegalDataException("CHAR types may only contain single byte characters (" + object + ")");
+                throw newIllegalDataException("CHAR types may only contain single byte characters (" + object + ")");
             }
             return len;
         }
@@ -248,7 +250,7 @@ public enum PDataType {
         @Override
         public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (!actualType.isCoercibleTo(this)) { // TODO: have isCoercibleTo that takes bytes, offset?
-                throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
+                throwConstraintViolationException(actualType,this);
             }
             if (length == 0) {
                 return null;
@@ -261,7 +263,7 @@ public enum PDataType {
             // TODO: UTF-8 decoder that will invert as it decodes
             String s = Bytes.toString(bytes, offset, length);
             if (length != s.length()) {
-               throw new IllegalDataException("CHAR types may only contain single byte characters (" + s + ")");
+               throw newIllegalDataException("CHAR types may only contain single byte characters (" + s + ")");
             }
             return s;
         }
@@ -288,7 +290,7 @@ public enum PDataType {
                 Integer actualMaxLength, Integer actualScale, SortOrder actualModifier,
                 Integer desiredMaxLength, Integer desiredScale, SortOrder expectedModifier) {
             if (o != null && actualType == PDataType.VARCHAR && ((String)o).length() != ptr.getLength()) {
-                throw new IllegalDataException("CHAR types may only contain single byte characters (" + o + ")");
+                throw newIllegalDataException("CHAR types may only contain single byte characters (" + o + ")");
             }
             super.coerceBytes(ptr, o, actualType, actualMaxLength, actualScale, actualModifier, desiredMaxLength, desiredScale, expectedModifier);
         }
@@ -332,10 +334,10 @@ public enum PDataType {
         @Override
         public Object toObject(String value) {
             if (value == null || value.length() == 0) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             if (StringUtil.hasMultiByteChars(value)) {
-                throw new IllegalDataException("CHAR types may only contain single byte characters (" + value + ")");
+                throw newIllegalDataException("CHAR types may only contain single byte characters (" + value + ")");
             }
             return value;
         }
@@ -376,7 +378,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] b, int o) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return this.getCodec().encodeLong(((Number)object).longValue(), b, o);
         }
@@ -406,7 +408,7 @@ public enum PDataType {
             case UNSIGNED_FLOAT:
                 Float f = (Float)object;
                 if (f > Long.MAX_VALUE || f < Long.MIN_VALUE) {
-                    throw new IllegalDataException(actualType + " value " + f + " cannot be cast to Long without changing its value");
+                    throw newIllegalDataException(actualType + " value " + f + " cannot be cast to Long without changing its value");
                 }
                 s = f.longValue();
                 return s;
@@ -414,7 +416,7 @@ public enum PDataType {
             case UNSIGNED_DOUBLE:
                 Double de = (Double) object;
                 if (de > Long.MAX_VALUE || de < Long.MIN_VALUE) {
-                    throw new IllegalDataException(actualType + " value " + de + " cannot be cast to Long without changing its value");
+                    throw newIllegalDataException(actualType + " value " + de + " cannot be cast to Long without changing its value");
                 }
                 s = de.longValue();
                 return s;
@@ -551,7 +553,7 @@ public enum PDataType {
             try {
                 return Long.parseLong(value);
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
 
@@ -576,7 +578,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] b, int o) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return this.getCodec().encodeInt(((Number)object).intValue(), b, o);
         }
@@ -589,7 +591,7 @@ public enum PDataType {
             }
             long l = (Long)o;
             if (l < Integer.MIN_VALUE || l > Integer.MAX_VALUE) {
-                throw new IllegalDataException(actualType + " value " + l + " cannot be cast to Integer without changing its value");
+                throw newIllegalDataException(actualType + " value " + l + " cannot be cast to Integer without changing its value");
             }
             int v = (int)l;
             return v;
@@ -689,7 +691,7 @@ public enum PDataType {
             try {
                 return Integer.parseInt(value);
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
 
@@ -739,7 +741,7 @@ public enum PDataType {
       @Override
       public int toBytes(Object object, byte[] bytes, int offset) {
         if (object == null) {
-          throw new ConstraintViolationException(this + " may not be null");
+          throw newIllegalDataException(this + " may not be null");
         }
         return this.getCodec().encodeShort(((Number)object).shortValue(), bytes, offset);
       }
@@ -752,7 +754,7 @@ public enum PDataType {
           }
           long l = (Long)o;
           if (l < Short.MIN_VALUE || l > Short.MAX_VALUE) {
-              throw new IllegalDataException(actualType + " value " + l + " cannot be cast to Short without changing its value");
+              throw newIllegalDataException(actualType + " value " + l + " cannot be cast to Short without changing its value");
           }
           short s = (short)l;
           return s;
@@ -793,7 +795,7 @@ public enum PDataType {
         try {
             return Short.parseShort(value);
         } catch (NumberFormatException e) {
-            throw new IllegalDataException(e);
+            throw newIllegalDataException(e);
         }
       }
       
@@ -873,7 +875,7 @@ public enum PDataType {
       @Override
       public int toBytes(Object object, byte[] bytes, int offset) {
         if (object == null) {
-          throw new ConstraintViolationException(this + " may not be null");
+          throw newIllegalDataException(this + " may not be null");
         }
         return this.getCodec().encodeByte(((Number)object).byteValue(), bytes, offset);
       }
@@ -887,7 +889,7 @@ public enum PDataType {
           Byte b = Byte.parseByte(value);
           return b;
         } catch (NumberFormatException e) {
-          throw new IllegalDataException(e);
+          throw newIllegalDataException(e);
         }
       }
       
@@ -899,7 +901,7 @@ public enum PDataType {
           }
           long l = (Long)o;
           if (l < Byte.MIN_VALUE || l > Byte.MAX_VALUE) {
-              throw new IllegalDataException(actualType + " value " + l + " cannot be cast to Byte without changing its value");
+              throw newIllegalDataException(actualType + " value " + l + " cannot be cast to Byte without changing its value");
           }
           return (byte)l;
       }
@@ -1008,7 +1010,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return this.getCodec().encodeFloat(((Number) object).floatValue(),
                     bytes, offset);
@@ -1022,7 +1024,7 @@ public enum PDataType {
             try {
                 return Float.parseFloat(value);
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
         
@@ -1044,7 +1046,7 @@ public enum PDataType {
                         || (d >= -Float.MAX_VALUE && d <= Float.MAX_VALUE)) {
                     return (float) d;
                 } else {
-                    throw new IllegalDataException(actualType + " value " + d + " cannot be cast to Float without changing its value");
+                    throw newIllegalDataException(actualType + " value " + d + " cannot be cast to Float without changing its value");
                 }
             case LONG:
             case UNSIGNED_LONG:
@@ -1188,7 +1190,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             } 
             return this.getCodec().encodeDouble(((Number) object).doubleValue(),
                     bytes, offset); 
@@ -1202,7 +1204,7 @@ public enum PDataType {
             try {
                 return Double.parseDouble(value);
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
         
@@ -1663,7 +1665,7 @@ public enum PDataType {
             try {
                 return new BigDecimal(value);
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
 
@@ -1692,7 +1694,7 @@ public enum PDataType {
         @Override
         public byte[] toBytes(Object object) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             byte[] bytes = new byte[getByteSize()];
             toBytes(object, bytes, 0);
@@ -1702,7 +1704,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             Timestamp value = (Timestamp)object;
             DATE.getCodec().encodeLong(value.getTime(), bytes, offset);
@@ -1984,7 +1986,7 @@ public enum PDataType {
         @Override
         public byte[] toBytes(Object object) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             byte[] bytes = new byte[getByteSize()];
             toBytes(object, bytes, 0);
@@ -1994,7 +1996,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             getCodec().encodeLong(((java.util.Date)object).getTime(), bytes, offset);
             return this.getByteSize();
@@ -2136,7 +2138,7 @@ public enum PDataType {
         @Override
         public byte[] toBytes(Object object) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             byte[] bytes = new byte[getByteSize()];
             toBytes(object, bytes, 0);
@@ -2146,7 +2148,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             Timestamp value = (Timestamp)object;
             UNSIGNED_DATE.getCodec().encodeLong(value.getTime(), bytes, offset);
@@ -2328,7 +2330,7 @@ public enum PDataType {
         @Override
         public byte[] toBytes(Object object) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             byte[] bytes = new byte[getByteSize()];
             toBytes(object, bytes, 0);
@@ -2338,7 +2340,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             getCodec().encodeLong(((java.util.Date)object).getTime(), bytes, offset);
             return this.getByteSize();
@@ -2452,7 +2454,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] b, int o) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return this.getCodec().encodeLong(((Number)object).longValue(), b, o);
         }
@@ -2514,11 +2516,11 @@ public enum PDataType {
             try {
                 Long l = Long.parseLong(value);
                 if (l.longValue() < 0) {
-                    throw new IllegalDataException("Value may not be negative(" + l + ")");
+                    throw newIllegalDataException("Value may not be negative(" + l + ")");
                 }
                 return l;
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
         
@@ -2553,7 +2555,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] b, int o) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return this.getCodec().encodeInt(((Number)object).intValue(), b, o);
         }
@@ -2610,11 +2612,11 @@ public enum PDataType {
             try {
                 Integer i = Integer.parseInt(value);
                 if (i.intValue() < 0) {
-                    throw new IllegalDataException("Value may not be negative(" + i + ")");
+                    throw newIllegalDataException("Value may not be negative(" + i + ")");
                 }
                 return i;
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
         
@@ -2657,7 +2659,7 @@ public enum PDataType {
       @Override
       public byte[] toBytes(Object object) {
         if (object == null) {
-          throw new ConstraintViolationException(this + " may not be null");
+          throw newIllegalDataException(this + " may not be null");
         }
         byte[] b = new byte[Bytes.SIZEOF_SHORT];
         toBytes(object, b, 0);
@@ -2667,7 +2669,7 @@ public enum PDataType {
       @Override
       public int toBytes(Object object, byte[] bytes, int offset) {
         if (object == null) {
-          throw new ConstraintViolationException(this + " may not be null");
+          throw newIllegalDataException(this + " may not be null");
         }
         return this.getCodec().encodeShort(((Number)object).shortValue(), bytes, offset);
       }
@@ -2680,11 +2682,11 @@ public enum PDataType {
         try {
           Short b = Short.parseShort(value);
           if (b.shortValue()<0) {
-              throw new IllegalDataException("Value may not be negative(" + b + ")");
+              throw newIllegalDataException("Value may not be negative(" + b + ")");
           }
           return b;
         } catch (NumberFormatException e) {
-          throw new IllegalDataException(e);
+          throw newIllegalDataException(e);
         }
       }
       
@@ -2763,7 +2765,7 @@ public enum PDataType {
       @Override
       public int toBytes(Object object, byte[] bytes, int offset) {
         if (object == null) {
-          throw new ConstraintViolationException(this + " may not be null");
+          throw newIllegalDataException(this + " may not be null");
         }
         return this.getCodec().encodeByte(((Number)object).byteValue(), bytes, offset);
       }
@@ -2776,11 +2778,11 @@ public enum PDataType {
         try {
           Byte b = Byte.parseByte(value);
           if (b.byteValue()<0) {
-              throw new IllegalDataException("Value may not be negative(" + b + ")");
+              throw newIllegalDataException("Value may not be negative(" + b + ")");
           }
           return b;
         } catch (NumberFormatException e) {
-          throw new IllegalDataException(e);
+          throw newIllegalDataException(e);
         }
       }
       
@@ -2861,7 +2863,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return this.getCodec().encodeFloat(((Number) object).floatValue(),
                     bytes, offset);
@@ -2875,12 +2877,12 @@ public enum PDataType {
             try {
                 Float f = Float.parseFloat(value);
                 if (f.floatValue() < 0) {
-                    throw new IllegalDataException("Value may not be negative("
+                    throw newIllegalDataException("Value may not be negative("
                             + f + ")");
                 }
                 return f;
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
         
@@ -2959,7 +2961,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             } 
             return this.getCodec().encodeDouble(((Number) object).doubleValue(),
                     bytes, offset); 
@@ -2973,12 +2975,12 @@ public enum PDataType {
             try {
                 Double d = Double.parseDouble(value);
                 if (d.doubleValue() < 0) {
-                    throw new IllegalDataException("Value may not be negative("
+                    throw newIllegalDataException("Value may not be negative("
                             + d + ")");
                 }
                 return d;
             } catch (NumberFormatException e) {
-                throw new IllegalDataException(e);
+                throw newIllegalDataException(e);
             }
         }
         
@@ -3023,7 +3025,7 @@ public enum PDataType {
         public byte[] toBytes(Object object) {
             if (object == null) {
                 // TODO: review - return null?
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return ((Boolean)object).booleanValue() ? TRUE_BYTES : FALSE_BYTES;
         }
@@ -3032,7 +3034,7 @@ public enum PDataType {
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
                 // TODO: review - return null?
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             bytes[offset] = ((Boolean)object).booleanValue() ? TRUE_BYTE : FALSE_BYTE;
             return BOOLEAN_LENGTH;
@@ -3042,7 +3044,7 @@ public enum PDataType {
         public byte[] toBytes(Object object, SortOrder sortOrder) {
             if (object == null) {
                 // TODO: review - return null?
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return ((Boolean)object).booleanValue() ^ sortOrder == SortOrder.ASC ? TRUE_BYTES : FALSE_BYTES;
         }
@@ -3056,7 +3058,7 @@ public enum PDataType {
             switch (actualType) {
                 case BOOLEAN:
                     if (length > 1) {
-                        throw new IllegalDataException("BOOLEAN may only be a single byte");
+                        throw newIllegalDataException("BOOLEAN may only be a single byte");
                     }
                     return ((bytes[offset] == FALSE_BYTE ^ sortOrder == SortOrder.DESC) ? Boolean.FALSE : Boolean.TRUE);
                 case DECIMAL:
@@ -3285,7 +3287,7 @@ public enum PDataType {
         @Override
         public byte[] toBytes(Object object) { // Deligate to VARBINARY
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return VARBINARY.toBytes(object);
         }
@@ -3293,7 +3295,7 @@ public enum PDataType {
         @Override
         public int toBytes(Object object, byte[] bytes, int offset) {
             if (object == null) {
-                throw new ConstraintViolationException(this + " may not be null");
+                throw newIllegalDataException(this + " may not be null");
             }
             return VARBINARY.toBytes(object, bytes, offset);
             
@@ -3311,7 +3313,7 @@ public enum PDataType {
         @Override
         public Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder, Integer maxLength, Integer scale) {
             if (!actualType.isCoercibleTo(this)) {
-                throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
+                throwConstraintViolationException(actualType, this);
             }
             return VARBINARY.toObject(bytes, offset, length, actualType, sortOrder);
         }
@@ -5744,6 +5746,33 @@ public enum PDataType {
         long diff = (long)d - l;
         return Long.signum(diff);
     }
+    
+    private static void checkForSufficientLength(byte[] b, int offset, int requiredLength) {
+        if (b.length < offset + requiredLength) {
+            throw new RuntimeException(new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA)
+            .setMessage("Expected length of at least " + requiredLength + " bytes, but had " + (b.length - offset)).build().buildException());
+        }
+    }
+    
+    private static Void throwConstraintViolationException(PDataType source, PDataType target) {
+        throw new ConstraintViolationException(new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CONVERT_TYPE)
+        .setMessage(source + " cannot be coerced to " + target).build().buildException());
+    }
+    
+    private static RuntimeException newIllegalDataException() {
+        return new IllegalDataException(new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA)
+        .build().buildException());
+    }
+
+    private static RuntimeException newIllegalDataException(String msg) {
+        return new IllegalDataException(new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA)
+        .setMessage(msg).build().buildException());
+    }
+
+    private static RuntimeException newIllegalDataException(Exception e) {
+        return new IllegalDataException(new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA)
+        .setRootCause(e).build().buildException());
+    }
 
     public static interface PDataCodec {
         public long decodeLong(ImmutableBytesWritable ptr, SortOrder sortOrder);
@@ -5894,31 +5923,26 @@ public enum PDataType {
 
         @Override
         public long decodeLong(byte[] bytes, int o, SortOrder sortOrder) {
-          Preconditions.checkNotNull(sortOrder);
-          long v = 0L;
-          byte b = bytes[o];
-          try {
+        	Preconditions.checkNotNull(sortOrder);
+        	checkForSufficientLength(bytes, o, Bytes.SIZEOF_LONG);
+            long v;
+            byte b = bytes[o];
             if (sortOrder == SortOrder.ASC) {
-              v = b ^ 0x80; // Flip sign bit back
-              for (int i = 1; i < Bytes.SIZEOF_LONG; i++) {
-                b = bytes[o + i];
-                v = (v << 8) + (b & 0xff);
-              }
+                v = b ^ 0x80; // Flip sign bit back
+                for (int i = 1; i < Bytes.SIZEOF_LONG; i++) {
+                    b = bytes[o + i];
+                    v = (v << 8) + (b & 0xff);
+                }
             } else {
-              b = (byte) (b ^ 0xff);
-              v = b ^ 0x80; // Flip sign bit back
-              for (int i = 1; i < Bytes.SIZEOF_LONG; i++) {
-                b = bytes[o + i];
-                b ^= 0xff;
-                v = (v << 8) + (b & 0xff);
-              }
-            }
-          } catch (RuntimeException e) {
-            if (e instanceof ArrayIndexOutOfBoundsException) {
-              throw new IllegalDataException("cannot coerced to data type LONG");
+                b = (byte)(b ^ 0xff);
+                v = b ^ 0x80; // Flip sign bit back
+                for (int i = 1; i < Bytes.SIZEOF_LONG; i++) {
+                    b = bytes[o + i];
+                    b ^= 0xff;
+                    v = (v << 8) + (b & 0xff);
+                }
             }
-          }
-          return v;
+            return v;
         }
 
 
@@ -5926,7 +5950,7 @@ public enum PDataType {
         public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
             long v = decodeLong(b, o, sortOrder);
             if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Integer without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Integer without changing its value");
             }
             return (int)v;
         }
@@ -5934,7 +5958,7 @@ public enum PDataType {
         @Override
         public int encodeFloat(float v, byte[] b, int o) {
             if (v < Long.MIN_VALUE || v > Long.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be encoded as an Long without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be encoded as an Long without changing its value");
             }
             return encodeLong((long)v, b, o);
         }
@@ -5942,13 +5966,14 @@ public enum PDataType {
         @Override
         public int encodeDouble(double v, byte[] b, int o) {
             if (v < Long.MIN_VALUE || v > Long.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be encoded as an Long without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be encoded as an Long without changing its value");
             }
             return encodeLong((long)v, b, o);
         }
 
         @Override
         public int encodeLong(long v, byte[] b, int o) {
+            checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
             b[o + 0] = (byte) ((v >> 56) ^ 0x80); // Flip sign bit so that INTEGER is binary comparable
             b[o + 1] = (byte) (v >> 48);
             b[o + 2] = (byte) (v >> 40);
@@ -5964,7 +5989,7 @@ public enum PDataType {
         public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
           long v = decodeLong(b, o, sortOrder);
           if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-              throw new IllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
+              throw newIllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
           }
           return (byte)v;
         }
@@ -5973,7 +5998,7 @@ public enum PDataType {
         public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
           long v = decodeLong(b, o, sortOrder);
           if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
-              throw new IllegalDataException("Value " + v + " cannot be cast to Short without changing its value");
+              throw newIllegalDataException("Value " + v + " cannot be cast to Short without changing its value");
           }
           return (short)v;
         }
@@ -6023,29 +6048,25 @@ public enum PDataType {
         @Override
         public int decodeInt(byte[] bytes, int o, SortOrder sortOrder) {
         	Preconditions.checkNotNull(sortOrder);
-            int v = 0;
-            try {
-              if (sortOrder == SortOrder.ASC) {
+            checkForSufficientLength(bytes, o, Bytes.SIZEOF_INT);
+            int v;
+            if (sortOrder == SortOrder.ASC) {
                 v = bytes[o] ^ 0x80; // Flip sign bit back
                 for (int i = 1; i < Bytes.SIZEOF_INT; i++) {
-                  v = (v << 8) + (bytes[o + i] & 0xff);
+                    v = (v << 8) + (bytes[o + i] & 0xff);
                 }
-              } else { 
+            } else { 
                 v = bytes[o] ^ 0xff ^ 0x80; // Flip sign bit back
                 for (int i = 1; i < Bytes.SIZEOF_INT; i++) {
-                  v = (v << 8) + ((bytes[o + i] ^ 0xff) & 0xff);
+                    v = (v << 8) + ((bytes[o + i] ^ 0xff) & 0xff);
                 }
-              }
-            } catch (RuntimeException e) {
-              if (e instanceof ArrayIndexOutOfBoundsException) {
-                throw new IllegalDataException("cannot coerced to data type INT");
-              }
             }
             return v;
         }
 
         @Override
         public int encodeInt(int v, byte[] b, int o) {
+            checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
             b[o + 0] = (byte) ((v >> 24) ^ 0x80); // Flip sign bit so that INTEGER is binary comparable
             b[o + 1] = (byte) (v >> 16);
             b[o + 2] = (byte) (v >> 8);
@@ -6056,7 +6077,7 @@ public enum PDataType {
         @Override
         public int encodeFloat(float v, byte[] b, int o) {
             if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be encoded as an Integer without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be encoded as an Integer without changing its value");
             }
             return encodeInt((int)v, b, o);
         }
@@ -6064,7 +6085,7 @@ public enum PDataType {
         @Override
         public int encodeDouble(double v, byte[] b, int o) {
             if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be encoded as an Integer without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be encoded as an Integer without changing its value");
             }
             return encodeInt((int)v, b, o);
         }
@@ -6072,7 +6093,7 @@ public enum PDataType {
         @Override
         public int encodeLong(long v, byte[] b, int o) {
             if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be encoded as an Integer without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be encoded as an Integer without changing its value");
             }
             return encodeInt((int)v,b,o);
         }
@@ -6081,7 +6102,7 @@ public enum PDataType {
         public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
           int v = decodeInt(b, o, sortOrder);
           if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-              throw new IllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
+              throw newIllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
           }
           return (byte)v;
         }
@@ -6090,7 +6111,7 @@ public enum PDataType {
         public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
           int v = decodeInt(b, o, sortOrder);
           if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
-              throw new IllegalDataException("Value " + v + " cannot be cast to Short without changing its value");
+              throw newIllegalDataException("Value " + v + " cannot be cast to Short without changing its value");
           }
           return (short)v;
         }
@@ -6135,37 +6156,33 @@ public enum PDataType {
       public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
         short v = decodeShort(b, o, sortOrder);
         if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-            throw new IllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
+            throw newIllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
         }
         return (byte)v;
       }
 
       @Override
       public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
-        Preconditions.checkNotNull(sortOrder);
-        int v = 0;
-        try {
-          if (sortOrder == SortOrder.ASC) {
+    	Preconditions.checkNotNull(sortOrder);
+        checkForSufficientLength(b, o, Bytes.SIZEOF_SHORT);
+        int v;
+        if (sortOrder == SortOrder.ASC) {
             v = b[o] ^ 0x80; // Flip sign bit back
             for (int i = 1; i < Bytes.SIZEOF_SHORT; i++) {
-              v = (v << 8) + (b[o + i] & 0xff);
+                v = (v << 8) + (b[o + i] & 0xff);
             }
-          } else {
+        } else {
             v = b[o] ^ 0xff ^ 0x80; // Flip sign bit back
             for (int i = 1; i < Bytes.SIZEOF_SHORT; i++) {
-              v = (v << 8) + ((b[o + i] ^ 0xff) & 0xff);
+                v = (v << 8) + ((b[o + i] ^ 0xff) & 0xff);
             }
-          }
-        } catch (RuntimeException e) {
-          if (e instanceof ArrayIndexOutOfBoundsException) {
-            throw new IllegalDataException("cannot coerced to data type SHORT");
-          }
         }
         return (short)v;
       }
       
       @Override
       public int encodeShort(short v, byte[] b, int o) {
+          checkForSufficientLength(b, o, Bytes.SIZEOF_SHORT);
           b[o + 0] = (byte) ((v >> 8) ^ 0x80); // Flip sign bit so that Short is binary comparable
           b[o + 1] = (byte) v;
           return Bytes.SIZEOF_SHORT;
@@ -6174,7 +6191,7 @@ public enum PDataType {
       @Override
       public int encodeLong(long v, byte[] b, int o) {
           if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
-              throw new IllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
+              throw newIllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
           }
           return encodeShort((short)v,b,o);
       }
@@ -6182,7 +6199,7 @@ public enum PDataType {
       @Override
       public int encodeInt(int v, byte[] b, int o) {
         if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
-          throw new IllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
+          throw newIllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
         }
         return encodeShort((short)v,b,o);
       }
@@ -6206,7 +6223,7 @@ public enum PDataType {
       @Override
       public int encodeDouble(double v, byte[] b, int o) {
           if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
-              throw new IllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
+              throw newIllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
           }
           return encodeShort((short)v,b,o);
       }
@@ -6214,7 +6231,7 @@ public enum PDataType {
       @Override
       public int encodeFloat(float v, byte[] b, int o) {
           if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
-              throw new IllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
+              throw newIllegalDataException("Value " + v + " cannot be encoded as an Short without changing its value");
           }
           return encodeShort((short)v,b,o);
       }
@@ -6248,6 +6265,7 @@ public enum PDataType {
       @Override
       public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
     	Preconditions.checkNotNull(sortOrder);
+        checkForSufficientLength(b, o, Bytes.SIZEOF_BYTE);
         int v;
         if (sortOrder == SortOrder.ASC) {
             v = b[o] ^ 0x80; // Flip sign bit back
@@ -6264,8 +6282,9 @@ public enum PDataType {
       
       @Override
       public int encodeShort(short v, byte[] b, int o) {
+          checkForSufficientLength(b, o, Bytes.SIZEOF_BYTE);
           if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-              throw new IllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
+              throw newIllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
           }
           return encodeByte((byte)v,b,o);
       }
@@ -6273,7 +6292,7 @@ public enum PDataType {
       @Override
       public int encodeLong(long v, byte[] b, int o) {
         if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-          throw new IllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
+          throw newIllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
         }
         return encodeByte((byte)v,b,o);
       }
@@ -6281,19 +6300,19 @@ public enum PDataType {
       @Override
       public int encodeInt(int v, byte[] b, int o) {
         if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-          throw new IllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
+          throw newIllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
         }
         return encodeByte((byte)v,b,o);
       }
       
       @Override
       public int encodeByte(byte v, byte[] b, int o) {
-        b[o] = (byte) (v ^ 0x80); // Flip sign bit so that Short is binary comparable
-        return Bytes.SIZEOF_BYTE;
+          checkForSufficientLength(b, o, Bytes.SIZEOF_BYTE);
+          b[o] = (byte) (v ^ 0x80); // Flip sign bit so that Short is binary comparable
+          return Bytes.SIZEOF_BYTE;
       }
         @Override
-        public double decodeDouble(byte[] b, int o,
-                SortOrder sortOrder) {
+        public double decodeDouble(byte[] b, int o, SortOrder sortOrder) {
             return decodeByte(b, o, sortOrder);
         }
 
@@ -6305,7 +6324,7 @@ public enum PDataType {
         @Override
         public int encodeFloat(float v, byte[] b, int o) {
             if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
             }
             return encodeByte((byte)v,b,o);
         }
@@ -6313,7 +6332,7 @@ public enum PDataType {
         @Override
         public int encodeDouble(double v, byte[] b, int o) {
             if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be encoded as an Byte without changing its value");
             }
             return encodeByte((byte)v,b,o);
         }
@@ -6335,13 +6354,14 @@ public enum PDataType {
 
       @Override
       public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
-        Preconditions.checkNotNull(sortOrder);
-        if (sortOrder == SortOrder.DESC) {
-          b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_BYTE], 0, Bytes.SIZEOF_BYTE);
-        }   
+    	Preconditions.checkNotNull(sortOrder);
+        checkForSufficientLength(b, o, Bytes.SIZEOF_BYTE);
         byte v = b[o];
+        if (sortOrder == SortOrder.DESC) {
+          v = SortOrder.invert(v);
+        }
         if (v < 0) {
-          throw new IllegalDataException();
+          throw newIllegalDataException();
         }
         return v;
       }
@@ -6349,7 +6369,7 @@ public enum PDataType {
       @Override
       public int encodeByte(byte v, byte[] b, int o) {
         if (v < 0) {
-          throw new IllegalDataException();
+          throw newIllegalDataException();
         }
         Bytes.putByte(b, o, v);
         return Bytes.SIZEOF_BYTE;
@@ -6364,34 +6384,30 @@ public enum PDataType {
         @Override
         public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
         	Preconditions.checkNotNull(sortOrder);
+            checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
             long v = 0;
-            try {
-              if (sortOrder == SortOrder.ASC) {
+            if (sortOrder == SortOrder.ASC) {
                 for(int i = o; i < o + Bytes.SIZEOF_LONG; i++) {
                   v <<= 8;
                   v ^= b[i] & 0xFF;
                 }
-              } else {
+            } else {
                 for(int i = o; i < o + Bytes.SIZEOF_LONG; i++) {
-                  v <<= 8;
-                  v ^= (b[i] & 0xFF) ^ 0xFF;
-                }
-              }
-            } catch (RuntimeException e) {
-              if (e instanceof ArrayIndexOutOfBoundsException) {
-                throw new IllegalDataException("cannot coerced to data type LONG");
-              }
+                    v <<= 8;
+                    v ^= (b[i] & 0xFF) ^ 0xFF;
+                  }
             }
             if (v < 0) {
-            	throw new IllegalDataException();
+                throw newIllegalDataException();
             }
             return v;
         }
 
         @Override
         public int encodeLong(long v, byte[] b, int o) {
+            checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
             if (v < 0) {
-                throw new IllegalDataException();
+                throw newIllegalDataException();
             }
             Bytes.putLong(b, o, v);
             return Bytes.SIZEOF_LONG;
@@ -6404,21 +6420,23 @@ public enum PDataType {
       
       @Override
       public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
-        Preconditions.checkNotNull(sortOrder);
-        if (sortOrder == SortOrder.DESC) {
-          b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_INT], 0, Bytes.SIZEOF_INT);
-        }
-        short v = Bytes.toShort(b, o);
-        if (v < 0) {
-          throw new IllegalDataException();
-        }
-        return v;
+    	  Preconditions.checkNotNull(sortOrder);
+          checkForSufficientLength(b, o, Bytes.SIZEOF_SHORT);
+          if (sortOrder == SortOrder.DESC) {
+              b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_SHORT], 0, Bytes.SIZEOF_SHORT);
+          }
+          short v = Bytes.toShort(b, o);
+          if (v < 0) {
+              throw newIllegalDataException();
+          }
+          return v;
       }
 
       @Override
       public int encodeShort(short v, byte[] b, int o) {
+          checkForSufficientLength(b, o, Bytes.SIZEOF_SHORT);
           if (v < 0) {
-              throw new IllegalDataException();
+              throw newIllegalDataException();
           }
           Bytes.putShort(b, o, v);
           return Bytes.SIZEOF_SHORT;
@@ -6432,21 +6450,23 @@ public enum PDataType {
 
         @Override
         public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
-          Preconditions.checkNotNull(sortOrder);
-          if (sortOrder == SortOrder.DESC) {
-            b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_INT], 0, Bytes.SIZEOF_INT);
-          }
-          int v = Bytes.toInt(b, o);
-          if (v < 0) {
-            throw new IllegalDataException();
-          }
-          return v;
+        	Preconditions.checkNotNull(sortOrder);
+            checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
+            if (sortOrder == SortOrder.DESC) {
+                b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_INT], 0, Bytes.SIZEOF_INT);
+            }
+            int v = Bytes.toInt(b, o);
+            if (v < 0) {
+                throw newIllegalDataException();
+            }
+            return v;
         }
 
         @Override
         public int encodeInt(int v, byte[] b, int o) {
+            checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
             if (v < 0) {
-                throw new IllegalDataException();
+                throw newIllegalDataException();
             }
             Bytes.putInt(b, o, v);
             return Bytes.SIZEOF_INT;
@@ -6462,7 +6482,7 @@ public enum PDataType {
         public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
             float v = decodeFloat(b, o, sortOrder);
             if (v < Long.MIN_VALUE || v > Long.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Long without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Long without changing its value");
             }
             return (long)v;
         }
@@ -6471,7 +6491,7 @@ public enum PDataType {
         public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
             float v = decodeFloat(b, o, sortOrder);
             if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Integer without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Integer without changing its value");
             }
             return (int) v;
         }
@@ -6480,7 +6500,7 @@ public enum PDataType {
         public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
             float v = decodeFloat(b, o, sortOrder);
             if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
             }
             return (byte) v;
         }
@@ -6489,7 +6509,7 @@ public enum PDataType {
         public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
             float v = decodeFloat(b, o, sortOrder);
             if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Short without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Short without changing its value");
             }
             return (short) v;
         }
@@ -6502,22 +6522,17 @@ public enum PDataType {
         
         @Override
         public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
-          Preconditions.checkNotNull(sortOrder);
-          try {
+        	Preconditions.checkNotNull(sortOrder);
+            checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
             if (sortOrder == SortOrder.DESC) {
-              for (int i = o; i < Bytes.SIZEOF_INT; i++) {
-                b[i] = (byte) (b[i] ^ 0xff);
-              }
-            }
-          } catch(RuntimeException e) {
-            if (e instanceof ArrayIndexOutOfBoundsException) {
-              throw new IllegalDataException("cannot coerced to data type FLOAT");
+                for (int i = o; i < Bytes.SIZEOF_INT; i++) {
+                    b[i] = (byte) (b[i] ^ 0xff);
+                }
             }
-          }
-          int i = Bytes.toInt(b, o);
-          i--;
-          i ^= (~i >> Integer.SIZE - 1) | Integer.MIN_VALUE;
-          return Float.intBitsToFloat(i);
+            int i = Bytes.toInt(b, o);
+            i--;
+            i ^= (~i >> Integer.SIZE - 1) | Integer.MIN_VALUE;
+            return Float.intBitsToFloat(i);
         }
         
         @Override
@@ -6547,17 +6562,18 @@ public enum PDataType {
                     || (v >= -Float.MAX_VALUE && v <= Float.MAX_VALUE)) {
                 return encodeFloat((float)v, b, o);
             } else {
-                throw new IllegalDataException("Value " + v + " cannot be encoded as an Float without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be encoded as an Float without changing its value");
             }
             
         }
         
         @Override
         public int encodeFloat(float v, byte[] b, int o) {
+            checkForSufficientLength(b, o, Bytes.SIZEOF_FLOAT);
             int i = Float.floatToIntBits(v);
             i = (i ^ ((i >> Integer.SIZE - 1) | Integer.MIN_VALUE)) + 1;
             Bytes.putInt(b, o, i);
-            return Bytes.SIZEOF_INT;
+            return Bytes.SIZEOF_FLOAT;
         }
         
         @Override
@@ -6580,7 +6596,7 @@ public enum PDataType {
         public long decodeLong(byte[] b, int o, SortOrder sortOrder) {
             double v = decodeDouble(b, o, sortOrder);
             if (v < Long.MIN_VALUE || v > Long.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Long without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Long without changing its value");
             }
             return (long) v;
         }
@@ -6589,7 +6605,7 @@ public enum PDataType {
         public int decodeInt(byte[] b, int o, SortOrder sortOrder) {
             double v = decodeDouble(b, o, sortOrder);
             if (v < Integer.MIN_VALUE || v > Integer.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Integer without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Integer without changing its value");
             }
             return (int) v;
         }
@@ -6598,7 +6614,7 @@ public enum PDataType {
         public byte decodeByte(byte[] b, int o, SortOrder sortOrder) {
             double v = decodeDouble(b, o, sortOrder);
             if (v < Byte.MIN_VALUE || v > Byte.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Byte without changing its value");
             }
             return (byte) v;
         }
@@ -6607,29 +6623,24 @@ public enum PDataType {
         public short decodeShort(byte[] b, int o, SortOrder sortOrder) {
             double v = decodeDouble(b, o, sortOrder);
             if (v < Short.MIN_VALUE || v > Short.MAX_VALUE) {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Short without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Short without changing its value");
             }
             return (short) v;
         }
         
         @Override
         public double decodeDouble(byte[] b, int o, SortOrder sortOrder) {
-          Preconditions.checkNotNull(sortOrder);
-          try {
+        	Preconditions.checkNotNull(sortOrder);
+            checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
             if (sortOrder == SortOrder.DESC) {
-              for (int i = o; i < Bytes.SIZEOF_LONG; i++) {
-                b[i] = (byte) (b[i] ^ 0xff);
-              }
+                for (int i = o; i < Bytes.SIZEOF_LONG; i++) {
+                    b[i] = (byte) (b[i] ^ 0xff);
+                }
             } 
-          } catch(RuntimeException e) {
-            if (e instanceof ArrayIndexOutOfBoundsException) {
-              throw new IllegalDataException("cannot coerced to data type DOUBLE");
-            }
-          }
-          long l = Bytes.toLong(b, o);
-          l--;
-          l ^= (~l >> Long.SIZE - 1) | Long.MIN_VALUE;
-          return Double.longBitsToDouble(l);
+            long l = Bytes.toLong(b, o);
+            l--;
+            l ^= (~l >> Long.SIZE - 1) | Long.MIN_VALUE;
+            return Double.longBitsToDouble(l);
         }
         
         @Override
@@ -6640,7 +6651,7 @@ public enum PDataType {
                     || (v >= -Float.MAX_VALUE && v <= Float.MAX_VALUE)) {
                 return (float) v;
             } else {
-                throw new IllegalDataException("Value " + v + " cannot be cast to Float without changing its value");
+                throw newIllegalDataException("Value " + v + " cannot be cast to Float without changing its value");
             }
             
         }
@@ -6667,6 +6678,7 @@ public enum PDataType {
         
         @Override
         public int encodeDouble(double v, byte[] b, int o) {
+            checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
             long l = Double.doubleToLongBits(v);
             l = (l ^ ((l >> Long.SIZE - 1) | Long.MIN_VALUE)) + 1;
             Bytes.putLong(b, o, l);
@@ -6695,8 +6707,9 @@ public enum PDataType {
         
         @Override
         public int encodeFloat(float v, byte[] b, int o) {
+            checkForSufficientLength(b, o, Bytes.SIZEOF_FLOAT);
             if (v < 0) {
-                throw new IllegalDataException();
+                throw newIllegalDataException();
             }
             Bytes.putFloat(b, o, v);
             return Bytes.SIZEOF_FLOAT;
@@ -6704,15 +6717,16 @@ public enum PDataType {
         
         @Override
         public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
-          Preconditions.checkNotNull(sortOrder);
-          if (sortOrder == SortOrder.DESC) {
-            b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_FLOAT], 0, Bytes.SIZEOF_FLOAT);
-          }
-          float v = Bytes.toFloat(b, o);
-          if (v < 0) {
-            throw new IllegalDataException();
-          }
-          return v;
+        	Preconditions.checkNotNull(sortOrder);
+            checkForSufficientLength(b, o, Bytes.SIZEOF_FLOAT);
+            if (sortOrder == SortOrder.DESC) {
+                b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_FLOAT], 0, Bytes.SIZEOF_FLOAT);
+            }
+            float v = Bytes.toFloat(b, o);
+            if (v < 0) {
+                throw newIllegalDataException();
+            }
+            return v;
         }
     }
     
@@ -6722,25 +6736,26 @@ public enum PDataType {
         
         @Override
         public int encodeDouble(double v, byte[] b, int o) {
+            checkForSufficientLength(b, o, Bytes.SIZEOF_DOUBLE);
             if (v < 0) {
-                throw new IllegalDataException();
+                throw newIllegalDataException();
             }
             Bytes.putDouble(b, o, v);
             return Bytes.SIZEOF_DOUBLE;
         }
         
         @Override
-        public double decodeDouble(byte[] b, int o,
-                SortOrder sortOrder) {
+        public double decodeDouble(byte[] b, int o, SortOrder sortOrder) {
         	Preconditions.checkNotNull(sortOrder);
-        	if (sortOrder == SortOrder.DESC) {
-        	  b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_DOUBLE], 0, Bytes.SIZEOF_DOUBLE);
-        	}
-        	double v = Bytes.toDouble(b, o);
-        	if (v < 0) {
-        	  throw new IllegalDataException();
-        	}
-        	return v;
+            checkForSufficientLength(b, o, Bytes.SIZEOF_DOUBLE);
+            if (sortOrder == SortOrder.DESC) {
+                b = SortOrder.invert(b, o, new byte[Bytes.SIZEOF_DOUBLE], 0, Bytes.SIZEOF_DOUBLE);
+            }
+            double v = Bytes.toDouble(b, o);
+            if (v < 0) {
+                throw newIllegalDataException();
+            }
+            return v;
         }
     }
 
@@ -7146,17 +7161,13 @@ public enum PDataType {
         coerceBytes(ptr, null, actualType, null, null, actualModifier, desiredMaxLength, null, expectedModifier);
     }
 
-    private static Void throwConstraintViolationException(PDataType source, PDataType target) {
-        throw new ConstraintViolationException(source + " cannot be coerced to " + target);
-    }
-    
     private static boolean isNonNegativeDate(java.util.Date date) {
         return (date == null || date.getTime() >= 0);
     }
     
     private static void throwIfNonNegativeDate(java.util.Date date) {
         if (!isNonNegativeDate(date)) {
-            throw new IllegalDataException("Value may not be negative(" + date + ")");
+            throw newIllegalDataException("Value may not be negative(" + date + ")");
         }
     }
     
@@ -7166,7 +7177,7 @@ public enum PDataType {
 
     private static void throwIfNonNegativeNumber(Number v) {
         if (!isNonNegativeNumber(v)) {
-            throw new IllegalDataException("Value may not be negative(" + v + ")");
+            throw newIllegalDataException("Value may not be negative(" + v + ")");
         }
     }
     
@@ -7278,7 +7289,7 @@ public enum PDataType {
         if (dataType != null) {
             return dataType;
         }
-        throw new IllegalDataException("Unsupported sql type: " + sqlTypeName);
+        throw newIllegalDataException("Unsupported sql type: " + sqlTypeName);
     }
     
     public static int sqlArrayType(String sqlTypeName) {
@@ -7320,7 +7331,7 @@ public enum PDataType {
 				return type;
 			}
 		}
-		throw new IllegalDataException("Unsupported sql type: " + typeId);
+		throw newIllegalDataException("Unsupported sql type: " + typeId);
 	}
 	
 	public static PhoenixArrayFactory[] getArrayFactory() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 4acb55c..8593246 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -545,7 +545,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             }
         } catch (SQLException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("ERROR 201 (22000): Illegal data."));
-            assertTrue(e.getCause().getMessage().contains("CHAR types may only contain single byte characters"));
+            assertTrue(e.getMessage().contains("CHAR types may only contain single byte characters"));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/test/java/org/apache/phoenix/schema/MappingTableDataTypeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/MappingTableDataTypeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/MappingTableDataTypeTest.java
deleted file mode 100644
index 64df35f..0000000
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/MappingTableDataTypeTest.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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.phoenix.schema;
-
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
-import static org.apache.phoenix.util.TestUtil.LOCALHOST;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.util.Properties;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.query.BaseTest;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-
-public class MappingTableDataTypeTest extends BaseTest{
-
-  private static final Log LOG = LogFactory.getLog(MappingTableDataTypeTest.class);
-
-  private static HBaseTestingUtility UTIL = null;
-  private static String URL = null;
-  private static HBaseAdmin admin = null;
-
-  @BeforeClass
-  public static void before() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    setUpConfigForMiniCluster(conf);
-    UTIL = new HBaseTestingUtility(conf);
-    UTIL.startMiniCluster(1);
-    String clientPort = UTIL.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
-    URL =
-        JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR + clientPort
-            + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
-    driver = initAndRegisterDriver(URL, ReadOnlyProps.EMPTY_PROPS);
-    admin = new HBaseAdmin(UTIL.getConfiguration());
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void testMappingHbaseTableToPhoenixTable() throws Exception {
-    final TableName tableName = TableName.valueOf("MTEST");
-    // Create table then get the single region for our new table.
-    HTable t = UTIL.createTable(tableName.getName(), Bytes.toBytes("cf"));
-    insertData(tableName.getName(), admin, t);
-    t.close();
-    try {
-      testCreateTableMismatchedType();
-    } catch (IllegalDataException e) {
-    }
-  }
-
-  private void insertData(final byte[] tableName, HBaseAdmin admin, HTable t) throws IOException,
-      InterruptedException {
-    Put p = new Put(Bytes.toBytes("row"));
-    p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("value1"));
-    t.put(p);
-    t.flushCommits();
-    admin.flush(tableName);
-  }
-
-  /**
-   * Test create a table in Phoenix with mismatched data type UNSIGNED_LONG
-   */
-  private void testCreateTableMismatchedType() throws Exception {
-    String ddl =
-        "create table IF NOT EXISTS MTEST (" + " id varchar NOT NULL primary key,"
-            + " \"cf\".\"q1\" unsigned_long" + " ) ";
-    Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    Connection conn = driver.connect(URL, props);
-    conn.createStatement().execute(ddl);
-    conn.commit();
-    String query = "select * from MTEST";
-    ResultSet rs = conn.createStatement().executeQuery(query);
-    rs.next();
-    rs.getLong(2);
-  }
-
-}
-
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeTest.java
index 7e9efcd..7c84928 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/PDataTypeTest.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.fail;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Date;
+import java.sql.SQLException;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
@@ -36,6 +37,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
@@ -1577,8 +1579,10 @@ public class PDataTypeTest {
         try {
             PDataType.UNSIGNED_DATE.toBytes(date1);
             fail();
-        } catch (IllegalDataException e) {
-            
+        } catch (RuntimeException e) {
+            assertTrue(e.getCause() instanceof SQLException);
+            SQLException sqlE = (SQLException)e.getCause();
+            assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), sqlE.getErrorCode());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ed8c82a0/phoenix-core/src/test/java/org/apache/phoenix/util/ColumnInfoTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/ColumnInfoTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/ColumnInfoTest.java
index 85f3b44..931d6fd 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/ColumnInfoTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/ColumnInfoTest.java
@@ -17,12 +17,14 @@
  */
 package org.apache.phoenix.util;
 
-import org.apache.phoenix.schema.IllegalDataException;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
+import java.sql.SQLException;
 import java.sql.Types;
 
-import static org.junit.Assert.assertEquals;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.junit.Test;
 
 public class ColumnInfoTest {
 
@@ -37,8 +39,14 @@ public class ColumnInfoTest {
         ColumnInfo.fromString("invalid");
     }
 
-    @Test(expected= IllegalDataException.class)
+    @Test
     public void testFromString_InvalidDataType() {
-        ColumnInfo.fromString("COLNAME:badType");
+        try {
+            ColumnInfo.fromString("COLNAME:badType");
+        } catch (RuntimeException e) {
+            assertTrue(e.getCause() instanceof SQLException);
+            SQLException sqlE = (SQLException)e.getCause();
+            assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), sqlE.getErrorCode());
+        }
     }
 }


[2/2] git commit: Remove unused imports and add required test annotations

Posted by ja...@apache.org.
Remove unused imports and add required test annotations


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/6d6926d2
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/6d6926d2
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/6d6926d2

Branch: refs/heads/master
Commit: 6d6926d270f45501f11d66688eedb657229b91e6
Parents: a2fbf48
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Oct 18 18:03:42 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Oct 18 19:41:20 2014 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/LastValueFunctionIT.java    | 13 +++++++++---
 .../apache/phoenix/end2end/LpadFunctionIT.java  |  2 ++
 .../phoenix/end2end/NthValueFunctionIT.java     | 13 +++++++++---
 .../org/apache/phoenix/end2end/ReadOnlyIT.java  |  3 ---
 .../phoenix/end2end/RegexpSplitFunctionIT.java  | 15 +++++++-------
 .../phoenix/end2end/RegexpSubstrFunctionIT.java | 21 +++++++++++---------
 .../phoenix/end2end/StatsCollectorIT.java       |  1 -
 7 files changed, 41 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d6926d2/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java
index 4886b4d..ff1751a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java
@@ -17,11 +17,18 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.junit.Assert.*;
-import java.sql.*;
-import static org.apache.hadoop.hbase.util.VersionInfo.getUrl;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d6926d2/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java
index cbc4ab7..2716f5e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LpadFunctionIT.java
@@ -29,12 +29,14 @@ import java.util.List;
 
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
 
 /**
  * Tests for the LPAD built-in function.
  */
+@Category(HBaseManagedTimeTest.class)
 public class LpadFunctionIT extends BaseHBaseManagedTimeIT {
     
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d6926d2/phoenix-core/src/it/java/org/apache/phoenix/end2end/NthValueFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NthValueFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NthValueFunctionIT.java
index 94067f9..1cf2643 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NthValueFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NthValueFunctionIT.java
@@ -17,11 +17,18 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.junit.Assert.*;
-import java.sql.*;
-import static org.apache.hadoop.hbase.util.VersionInfo.getUrl;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class NthValueFunctionIT extends BaseHBaseManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d6926d2/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadOnlyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadOnlyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadOnlyIT.java
index 515acac..a425d3a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadOnlyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadOnlyIT.java
@@ -18,7 +18,6 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -26,10 +25,8 @@ import static org.junit.Assert.fail;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
-import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Properties;
-import java.sql.ResultSetMetaData;
 
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d6926d2/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSplitFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSplitFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSplitFunctionIT.java
index 73f1828..0bda438 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSplitFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSplitFunctionIT.java
@@ -17,8 +17,11 @@
  */
 package org.apache.phoenix.end2end;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.sql.Array;
 import java.sql.Connection;
@@ -28,12 +31,8 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Types;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 @Category(HBaseManagedTimeTest.class)
 public class RegexpSplitFunctionIT extends BaseHBaseManagedTimeIT {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d6926d2/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
index 1121207..be4646b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
@@ -17,19 +17,22 @@
  */
 package org.apache.phoenix.end2end;
 
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.PropertiesUtil;
+import static org.apache.phoenix.util.TestUtil.GROUPBYTEST_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.sql.*;
-import java.util.Properties;
-
-import static org.apache.phoenix.util.TestUtil.GROUPBYTEST_NAME;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.*;
-
 @Category(HBaseManagedTimeTest.class)
 public class RegexpSubstrFunctionIT extends BaseHBaseManagedTimeIT {
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d6926d2/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index 49def02..b48e260 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -74,7 +74,6 @@ public class StatsCollectorIT extends BaseOwnClusterHBaseManagedTimeIT {
         Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
-        long ts = nextTimestamp();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);