You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by pr...@apache.org on 2012/08/10 01:18:00 UTC

svn commit: r1371530 - /incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/

Author: prestonc
Date: Thu Aug  9 23:17:59 2012
New Revision: 1371530

URL: http://svn.apache.org/viewvc?rev=1371530&view=rev
Log:
Added all the derived types of integer for casting. Also update the string to float and double to reduce the number of computations.

Added:
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToByteOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToLongOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNegativeIntegerOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonNegativeIntegerOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonPositiveIntegerOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToPositiveIntegerOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToShortOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedByteOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedIntOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedLongOperation.java   (with props)
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedShortOperation.java   (with props)
Modified:
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/AbstractCastToOperation.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastScalarEvaluatorFactory.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntegerOperation.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/AbstractCastToOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/AbstractCastToOperation.java?rev=1371530&r1=1371529&r2=1371530&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/AbstractCastToOperation.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/AbstractCastToOperation.java Thu Aug  9 23:17:59 2012
@@ -14,13 +14,19 @@ import org.apache.vxquery.exceptions.Err
 import org.apache.vxquery.exceptions.SystemException;
 
 import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public abstract class AbstractCastToOperation {
+    
+    /**
+     * Primitive Datatypes
+     */
     public void convertAnyURI(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
         throw new SystemException(ErrorCode.XPTY0004);
     }
@@ -112,4 +118,56 @@ public abstract class AbstractCastToOper
     public void convertYMDuration(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
         throw new SystemException(ErrorCode.XPTY0004);
     }
+    
+    /**
+     * Derived Datatypes
+     */
+    public void convertByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertNonNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertNonPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertUnsignedByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertUnsignedInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertUnsignedLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+    
+    public void convertUnsignedShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        throw new SystemException(ErrorCode.XPTY0004);
+    }
+
 }

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastScalarEvaluatorFactory.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastScalarEvaluatorFactory.java?rev=1371530&r1=1371529&r2=1371530&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastScalarEvaluatorFactory.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastScalarEvaluatorFactory.java Thu Aug  9 23:17:59 2012
@@ -39,10 +39,12 @@ import edu.uci.ics.hyracks.algebricks.ru
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.data.std.api.IPointable;
 import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 
@@ -74,6 +76,9 @@ public class CastScalarEvaluatorFactory 
                 abvs.reset();
                 try {
                     switch (tid) {
+                    /**
+                     * Primitive Datatypes (Alphabetical)
+                     */
                         case ValueTag.XS_ANY_URI_TAG:
                             tvp.getValue(tp.utf8sp);
                             aOp.convertAnyURI(tp.utf8sp, dOut);
@@ -212,6 +217,81 @@ public class CastScalarEvaluatorFactory 
                             result.set(abvs);
                             return;
 
+                            /**
+                             * Derived Datatypes (Alphabetical)
+                             */
+                        case ValueTag.XS_BYTE_TAG:
+                            tvp.getValue(tp.bytep);
+                            aOp.convertByte(tp.bytep, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_INT_TAG:
+                            tvp.getValue(tp.intp);
+                            aOp.convertInt(tp.intp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_LONG_TAG:
+                            tvp.getValue(tp.longp);
+                            aOp.convertLong(tp.longp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_NEGATIVE_INTEGER_TAG:
+                            tvp.getValue(tp.longp);
+                            aOp.convertNegativeInteger(tp.longp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_NON_NEGATIVE_INTEGER_TAG:
+                            tvp.getValue(tp.longp);
+                            aOp.convertNonNegativeInteger(tp.longp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_NON_POSITIVE_INTEGER_TAG:
+                            tvp.getValue(tp.longp);
+                            aOp.convertNonPositiveInteger(tp.longp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_POSITIVE_INTEGER_TAG:
+                            tvp.getValue(tp.longp);
+                            aOp.convertPositiveInteger(tp.longp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_SHORT_TAG:
+                            tvp.getValue(tp.shortp);
+                            aOp.convertShort(tp.shortp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_UNSIGNED_BYTE_TAG:
+                            tvp.getValue(tp.bytep);
+                            aOp.convertUnsignedByte(tp.bytep, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_UNSIGNED_INT_TAG:
+                            tvp.getValue(tp.intp);
+                            aOp.convertUnsignedInt(tp.intp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_UNSIGNED_LONG_TAG:
+                            tvp.getValue(tp.longp);
+                            aOp.convertUnsignedLong(tp.longp, dOut);
+                            result.set(abvs);
+                            return;
+
+                        case ValueTag.XS_UNSIGNED_SHORT_TAG:
+                            tvp.getValue(tp.shortp);
+                            aOp.convertUnsignedShort(tp.shortp, dOut);
+                            result.set(abvs);
+                            return;
+
                     }
                 } catch (SystemException se) {
                     throw se;
@@ -269,6 +349,32 @@ public class CastScalarEvaluatorFactory 
                     aOp = new CastToUntypedAtomicOperation();
                 } else if (sType.getItemType() == BuiltinTypeRegistry.XS_YEAR_MONTH_DURATION) {
                     aOp = new CastToYMDurationOperation();
+
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_BYTE) {
+                    aOp = new CastToByteOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_INT) {
+                    aOp = new CastToIntOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_LONG) {
+                    aOp = new CastToLongOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_NEGATIVE_INTEGER) {
+                    aOp = new CastToNegativeIntegerOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_NON_NEGATIVE_INTEGER) {
+                    aOp = new CastToNonNegativeIntegerOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_NON_POSITIVE_INTEGER) {
+                    aOp = new CastToNonPositiveIntegerOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_POSITIVE_INTEGER) {
+                    aOp = new CastToPositiveIntegerOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_SHORT) {
+                    aOp = new CastToShortOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_UNSIGNED_BYTE) {
+                    aOp = new CastToUnsignedByteOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_UNSIGNED_INT) {
+                    aOp = new CastToUnsignedIntOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_UNSIGNED_LONG) {
+                    aOp = new CastToUnsignedLongOperation();
+                } else if (sType.getItemType() == BuiltinTypeRegistry.XS_UNSIGNED_SHORT) {
+                    aOp = new CastToUnsignedShortOperation();
+
                 } else {
                     aOp = new CastToUntypedAtomicOperation();
                 }
@@ -277,6 +383,7 @@ public class CastScalarEvaluatorFactory 
             private int getBaseTypeForCasts(int tid) throws SystemException {
                 while (true) {
                     switch (tid) {
+                    // Primitive Datatypes
                         case ValueTag.XS_ANY_URI_TAG:
                         case ValueTag.XS_BASE64_BINARY_TAG:
                         case ValueTag.XS_BOOLEAN_TAG:
@@ -299,6 +406,19 @@ public class CastScalarEvaluatorFactory 
                         case ValueTag.XS_TIME_TAG:
                         case ValueTag.XS_UNTYPED_ATOMIC_TAG:
                         case ValueTag.XS_YEAR_MONTH_DURATION_TAG:
+                            // Derived Datatypes
+                        case ValueTag.XS_BYTE_TAG:
+                        case ValueTag.XS_INT_TAG:
+                        case ValueTag.XS_LONG_TAG:
+                        case ValueTag.XS_NEGATIVE_INTEGER_TAG:
+                        case ValueTag.XS_NON_NEGATIVE_INTEGER_TAG:
+                        case ValueTag.XS_NON_POSITIVE_INTEGER_TAG:
+                        case ValueTag.XS_POSITIVE_INTEGER_TAG:
+                        case ValueTag.XS_SHORT_TAG:
+                        case ValueTag.XS_UNSIGNED_BYTE_TAG:
+                        case ValueTag.XS_UNSIGNED_INT_TAG:
+                        case ValueTag.XS_UNSIGNED_LONG_TAG:
+                        case ValueTag.XS_UNSIGNED_SHORT_TAG:
                             return tid;
 
                         case ValueTag.XS_ANY_ATOMIC_TAG:
@@ -315,6 +435,8 @@ public class CastScalarEvaluatorFactory 
 
     private static class TypedPointables {
         BooleanPointable boolp = (BooleanPointable) BooleanPointable.FACTORY.createPointable();
+        BytePointable bytep = (BytePointable) BytePointable.FACTORY.createPointable();
+        ShortPointable shortp = (ShortPointable) ShortPointable.FACTORY.createPointable();
         IntegerPointable intp = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
         LongPointable longp = (LongPointable) LongPointable.FACTORY.createPointable();
         FloatPointable floatp = (FloatPointable) FloatPointable.FACTORY.createPointable();

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToByteOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToByteOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToByteOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToByteOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,174 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.vxquery.datamodel.accessors.atomic.XSDecimalPointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.runtime.functions.strings.ICharacterIterator;
+import org.apache.vxquery.runtime.functions.strings.UTF8StringCharacterIterator;
+
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class CastToByteOperation extends AbstractCastToOperation {
+    boolean negativeAllowed = true;
+    int returnTag = ValueTag.XS_BYTE_TAG;
+
+    @Override
+    public void convertBoolean(BooleanPointable boolp, DataOutput dOut) throws SystemException, IOException {
+        dOut.write(returnTag);
+        dOut.write((byte) (boolp.getBoolean() ? 1 : 0));
+    }
+
+    @Override
+    public void convertDecimal(XSDecimalPointable decp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(decp, dOut);
+    }
+
+    @Override
+    public void convertDouble(DoublePointable doublep, DataOutput dOut) throws SystemException, IOException {
+        double doubleValue = doublep.getDouble();
+        if (Double.isInfinite(doubleValue) || Double.isNaN(doubleValue)) {
+            throw new SystemException(ErrorCode.FOCA0002);
+        }
+        if (doubleValue > Byte.MAX_VALUE || doubleValue < Byte.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FOCA0003);
+        }
+        if (doublep.byteValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        dOut.write(returnTag);
+        dOut.write(doublep.byteValue());
+    }
+
+    @Override
+    public void convertFloat(FloatPointable floatp, DataOutput dOut) throws SystemException, IOException {
+        float floatValue = floatp.getFloat();
+        if (Float.isInfinite(floatValue) || Float.isNaN(floatValue)) {
+            throw new SystemException(ErrorCode.FOCA0002);
+        }
+        if (floatValue > Byte.MAX_VALUE || floatValue < Byte.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FOCA0003);
+        }
+        if (floatp.byteValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        dOut.write(returnTag);
+        dOut.write(floatp.byteValue());
+    }
+
+    @Override
+    public void convertInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(longp, dOut);
+    }
+
+    @Override
+    public void convertString(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
+        ICharacterIterator charIterator = new UTF8StringCharacterIterator(stringp);
+        charIterator.reset();
+        long value = 0;
+        int c = 0;
+        boolean negative = false;
+
+        // Check the first character.
+        c = charIterator.next();
+        if (c == Character.valueOf('-') && negativeAllowed) {
+            negative = true;
+            c = charIterator.next();
+        }
+
+        // Read the numeric value.
+        do {
+            if (Character.isDigit(c)) {
+                value = value * 10 + Character.getNumericValue(c);
+            } else {
+                throw new SystemException(ErrorCode.FORG0001);
+            }
+        } while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR);
+
+        if (value > Byte.MAX_VALUE || value < Byte.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+
+        dOut.write(returnTag);
+        dOut.write((byte) (negative ? -value : value));
+    }
+
+    @Override
+    public void convertUntypedAtomic(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
+        convertString(stringp, dOut);
+    }
+
+    /**
+     * Derived Datatypes
+     */
+    public void convertByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(bytep, dOut);
+    }
+
+    public void convertInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(intp, dOut);
+    }
+
+    public void convertLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(longp, dOut);
+    }
+
+    public void convertNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(longp, dOut);
+    }
+
+    public void convertNonNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(longp, dOut);
+    }
+
+    public void convertNonPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(longp, dOut);
+    }
+
+    public void convertPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(longp, dOut);
+    }
+
+    public void convertShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(shortp, dOut);
+    }
+
+    public void convertUnsignedByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(bytep, dOut);
+    }
+
+    public void convertUnsignedInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(intp, dOut);
+    }
+
+    public void convertUnsignedLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(longp, dOut);
+    }
+
+    public void convertUnsignedShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeByteValue(shortp, dOut);
+    }
+
+    private void writeByteValue(INumeric numericp, DataOutput dOut) throws SystemException, IOException {
+        if (numericp.longValue() > Byte.MAX_VALUE || numericp.longValue() < Byte.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        if (numericp.byteValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+
+        dOut.write(returnTag);
+        dOut.write(numericp.byteValue());
+    }
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToByteOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java?rev=1371530&r1=1371529&r2=1371530&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDecimalOperation.java Thu Aug  9 23:17:59 2012
@@ -10,13 +10,20 @@ import org.apache.vxquery.exceptions.Sys
 import org.apache.vxquery.runtime.functions.strings.ICharacterIterator;
 import org.apache.vxquery.runtime.functions.strings.UTF8StringCharacterIterator;
 
+import edu.uci.ics.hyracks.data.std.api.INumeric;
 import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 
 public class CastToDecimalOperation extends AbstractCastToOperation {
+    private ArrayBackedValueStorage abvsInner = new ArrayBackedValueStorage();
+    private DataOutput dOutInner = abvsInner.getDataOutput();
 
     @Override
     public void convertBoolean(BooleanPointable boolp, DataOutput dOut) throws SystemException, IOException {
@@ -34,47 +41,24 @@ public class CastToDecimalOperation exte
 
     @Override
     public void convertDouble(DoublePointable doublep, DataOutput dOut) throws SystemException, IOException {
-        double doubleValue = doublep.getDouble();
-        byte decimalPlace = 0;
-        // Move the decimal
-        while (doubleValue % 1 != 0 && (doubleValue != 0 || doubleValue != -0)) {
-            if (decimalPlace + 1 > XSDecimalPointable.PRECISION) {
-                throw new SystemException(ErrorCode.FOCA0001);
-            }
-            decimalPlace++;
-            doubleValue *= 10;
-        }
-        // Remove extra zeros
-        while (doubleValue % 10 == 0 && (doubleValue != 0 || doubleValue != -0)) {
-            doubleValue /= 10;
-            --decimalPlace;
-        }
-        dOut.write(ValueTag.XS_DECIMAL_TAG);
-        dOut.write(decimalPlace);
-        dOut.writeLong((long) doubleValue);
+        abvsInner.reset();
+        CastToStringOperation castTo = new CastToStringOperation();
+        castTo.convertDoubleCanonical(doublep, dOutInner);
+
+        UTF8StringPointable stringp = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
+        stringp.set(abvsInner.getByteArray(), abvsInner.getStartOffset() + 1, abvsInner.getLength() - 1);
+        convertString(stringp, dOut);
     }
 
     @Override
     public void convertFloat(FloatPointable floatp, DataOutput dOut) throws SystemException, IOException {
-        float floatValue = floatp.getFloat();
-        byte decimalPlace = 0;
-        
-        // Move the decimal
-        while (floatValue % 1 != 0 && (floatValue != 0 || floatValue != -0)) {
-            if (decimalPlace + 1 > XSDecimalPointable.PRECISION) {
-                throw new SystemException(ErrorCode.FOCA0001);
-            }
-            decimalPlace++;
-            floatValue *= 10;
-        }
-        // Remove extra zeros
-        while (floatValue % 10 == 0 && (floatValue != 0 || floatValue != -0)) {
-            floatValue /= 10;
-            --decimalPlace;
-        }
-        dOut.write(ValueTag.XS_DECIMAL_TAG);
-        dOut.write(decimalPlace);
-        dOut.writeLong((long) floatValue);
+        abvsInner.reset();
+        CastToStringOperation castTo = new CastToStringOperation();
+        castTo.convertFloatCanonical(floatp, dOutInner);
+
+        UTF8StringPointable stringp = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable();
+        stringp.set(abvsInner.getByteArray(), abvsInner.getStartOffset() + 1, abvsInner.getLength() - 1);
+        convertString(stringp, dOut);
     }
 
     @Override
@@ -94,7 +78,15 @@ public class CastToDecimalOperation exte
         int count = 0;
         int c = 0;
 
-        while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR) {
+        // Check sign.
+        c = charIterator.next();
+        if (c == Character.valueOf('-')) {
+            negativeValue = true;
+            c = charIterator.next();
+        }
+
+        // Read in the number.
+        do {
             if (count + 1 > XSDecimalPointable.PRECISION) {
                 throw new SystemException(ErrorCode.FOCA0006);
             } else if (Character.isDigit(c)) {
@@ -103,16 +95,34 @@ public class CastToDecimalOperation exte
                     decimalPlace++;
                 }
                 count++;
-            } else if (c == Character.valueOf('-')) {
-                negativeValue = true;
-            } else if (c == Character.valueOf('.')) {
+            } else if (c == Character.valueOf('.') && pastDecimal == false) {
                 pastDecimal = true;
+            } else if (c == Character.valueOf('E') || c == Character.valueOf('e')) {
+                break;
             } else {
                 throw new SystemException(ErrorCode.FORG0001);
             }
-        }
-        if (negativeValue) {
-            value *= -1;
+        } while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR);
+
+        // Parse the exponent.
+        if (c == Character.valueOf('E') || c == Character.valueOf('e')) {
+            int moveOffset = 0;
+            boolean negativeOffset = false;
+            // Check for the negative sign.
+            c = charIterator.next();
+            if (c == Character.valueOf('-')) {
+                negativeOffset = true;
+                c = charIterator.next();
+            }
+            // Process the numeric value.
+            do {
+                if (Character.isDigit(c)) {
+                    moveOffset = moveOffset * 10 + Character.getNumericValue(c);
+                } else {
+                    throw new SystemException(ErrorCode.FORG0001);
+                }
+            } while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR);
+            decimalPlace -= (negativeOffset ? -moveOffset : moveOffset);
         }
 
         // Normalize the value and take off trailing zeros.
@@ -120,9 +130,13 @@ public class CastToDecimalOperation exte
             value /= 10;
             --decimalPlace;
         }
+        if (decimalPlace > XSDecimalPointable.PRECISION) {
+            throw new SystemException(ErrorCode.FOCA0006);
+        }
+
         dOut.write(ValueTag.XS_DECIMAL_TAG);
         dOut.write(decimalPlace);
-        dOut.writeLong(value);
+        dOut.writeLong((negativeValue ? -value : value));
     }
 
     @Override
@@ -130,4 +144,61 @@ public class CastToDecimalOperation exte
         convertString(stringp, dOut);
     }
 
+    /**
+     * Derived Datatypes
+     */
+    public void convertByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(bytep, dOut);
+    }
+
+    public void convertInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(intp, dOut);
+    }
+
+    public void convertLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(longp, dOut);
+    }
+
+    public void convertNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(longp, dOut);
+    }
+
+    public void convertNonNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(longp, dOut);
+    }
+
+    public void convertNonPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(longp, dOut);
+    }
+
+    public void convertPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(longp, dOut);
+    }
+
+    public void convertShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(shortp, dOut);
+    }
+
+    public void convertUnsignedByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(bytep, dOut);
+    }
+
+    public void convertUnsignedInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(intp, dOut);
+    }
+
+    public void convertUnsignedLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(longp, dOut);
+    }
+
+    public void convertUnsignedShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeDecimalValue(shortp, dOut);
+    }
+
+    private void writeDecimalValue(INumeric numericp, DataOutput dOut) throws SystemException, IOException {
+        dOut.write(ValueTag.XS_DECIMAL_TAG);
+        dOut.write((byte) 0);
+        dOut.writeLong(numericp.longValue());
+    }
+
 }
\ No newline at end of file

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java?rev=1371530&r1=1371529&r2=1371530&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToDoubleOperation.java Thu Aug  9 23:17:59 2012
@@ -10,13 +10,23 @@ import org.apache.vxquery.exceptions.Sys
 import org.apache.vxquery.runtime.functions.strings.ICharacterIterator;
 import org.apache.vxquery.runtime.functions.strings.UTF8StringCharacterIterator;
 
+import edu.uci.ics.hyracks.data.std.api.INumeric;
 import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class CastToDoubleOperation extends AbstractCastToOperation {
+    /*
+     * All the positive powers of 10 that can be represented exactly in float.
+     */
+    private static final double powersOf10[] = { 1.0e0, 1.0e1, 1.0e2, 1.0e3, 1.0e4, 1.0e5, 1.0e6, 1.0e7, 1.0e8, 1.0e9,
+            1.0e10, 1.0e11, 1.0e12, 1.0e13, 1.0e14, 1.0e15, 1.0e16, 1.0e17, 1.0e18, 1.0e19, 1.0e20, 1.0e21, 1.0e22 };
+    private static final double powersOf2[] = { 1.0e16d, 1.0e32f, 1.0e64, 1.0e128, 1.0e256 };
 
     @Override
     public void convertBoolean(BooleanPointable boolp, DataOutput dOut) throws SystemException, IOException {
@@ -56,7 +66,7 @@ public class CastToDoubleOperation exten
     public void convertString(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
         ICharacterIterator charIterator = new UTF8StringCharacterIterator(stringp);
         charIterator.reset();
-        byte decimalPlace = 0;
+        short decimalPlace = 0;
         long value = 0;
         double valueDouble;
         boolean pastDecimal = false, negativeValue = false;
@@ -127,19 +137,73 @@ public class CastToDoubleOperation exten
                 }
             }
 
-            // TODO Verify the long value and exponent are combined to give the correct double.
+            /*
+             * The following conditions to create the floating point value is using known valid float values.
+             * In addition, each one only needs one or two operations to get the float value, further minimizing
+             * possible errors. (Not perfect, but pretty good.)
+             */
             valueDouble = (double) value;
-            while (decimalPlace != 0 && valueDouble != 0) {
-                if (decimalPlace > 0) {
-                    --decimalPlace;
-                    valueDouble *= 10;
+            if (decimalPlace == 0 || valueDouble == 0.0f) {
+                // No modification required to float value.
+            } else if (decimalPlace >= 0) {
+                if (decimalPlace <= 16) {
+                    valueDouble *= powersOf10[decimalPlace];
                 } else {
-                    ++decimalPlace;
-                    valueDouble /= 10;
+                    // Multiply the value based on the exponent binary.
+                    if ((decimalPlace & 15) != 0) {
+                        valueDouble *= powersOf10[decimalPlace & 15];
+                    }
+                    if ((decimalPlace >>= 4) != 0) {
+                        int j;
+                        for (j = 0; decimalPlace > 1; j++, decimalPlace >>= 1) {
+                            if ((decimalPlace & 1) != 0)
+                                valueDouble *= powersOf2[j];
+                        }
+                        // Handle the last cast for infinity and max value.
+                        double t = valueDouble * powersOf2[j];
+                        if (Double.isInfinite(t)) {
+                            // Overflow
+                            t = valueDouble / 2.0;
+                            t *= powersOf2[j];
+                            if (Double.isInfinite(t)) {
+                                valueDouble = Double.POSITIVE_INFINITY;
+                            }
+                            t = Double.MAX_VALUE;
+                        }
+                        valueDouble = t;
+                    }
+                }
+            } else {
+                if (decimalPlace >= -16) {
+                    valueDouble /= powersOf10[-decimalPlace];
+                } else {
+                    if ((decimalPlace & 15) != 0) {
+                        valueDouble /= powersOf10[decimalPlace & 15];
+                    }
+                    if ((decimalPlace >>= 4) != 0) {
+                        int j;
+                        for (j = 0; decimalPlace > 1; j++, decimalPlace >>= 1) {
+                            if ((decimalPlace & 1) != 0)
+                                valueDouble /= powersOf2[j];
+                        }
+                        // Handle the last cast for zero and min value.
+                        double t = valueDouble / powersOf2[j];
+                        if (t == 0.0) {
+                            // Underflow.
+                            t = valueDouble * 2.0;
+                            t /= powersOf2[j];
+                            if (t == 0.0) {
+                                valueDouble = 0.0;
+                            }
+                            t = Double.MIN_VALUE;
+                        }
+                        valueDouble = t;
+                    }
                 }
             }
 
         }
+
         dOut.write(ValueTag.XS_DOUBLE_TAG);
         dOut.writeDouble((negativeValue ? -valueDouble : valueDouble));
     }
@@ -149,4 +213,60 @@ public class CastToDoubleOperation exten
         convertString(stringp, dOut);
     }
 
+    /**
+     * Derived Datatypes
+     */
+    public void convertByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(bytep, dOut);
+    }
+
+    public void convertInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(intp, dOut);
+    }
+
+    public void convertLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertNonNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertNonPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(shortp, dOut);
+    }
+
+    public void convertUnsignedByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(bytep, dOut);
+    }
+
+    public void convertUnsignedInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(intp, dOut);
+    }
+
+    public void convertUnsignedLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertUnsignedShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(shortp, dOut);
+    }
+
+    private void writeDoubleValue(INumeric numericp, DataOutput dOut) throws SystemException, IOException {
+        dOut.write(ValueTag.XS_DOUBLE_TAG);
+        dOut.writeDouble(numericp.doubleValue());
+    }
+
 }
\ No newline at end of file

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java?rev=1371530&r1=1371529&r2=1371530&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToFloatOperation.java Thu Aug  9 23:17:59 2012
@@ -10,13 +10,24 @@ import org.apache.vxquery.exceptions.Sys
 import org.apache.vxquery.runtime.functions.strings.ICharacterIterator;
 import org.apache.vxquery.runtime.functions.strings.UTF8StringCharacterIterator;
 
+import edu.uci.ics.hyracks.data.std.api.INumeric;
 import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class CastToFloatOperation extends AbstractCastToOperation {
+    /*
+     * All the positive powers of 10 that can be represented exactly in float.
+     */
+    private static final float powersOf10upTo10[] = { 1.0e0f, 1.0e1f, 1.0e2f, 1.0e3f, 1.0e4f, 1.0e5f, 1.0e6f, 1.0e7f,
+            1.0e8f, 1.0e9f, 1.0e10f };
+    private static final float powersOf10from20to30[] = { 1.0e20f, 1.0e21f, 1.0e22f, 1.0e23f, 1.0e24f, 1.0e25f,
+            1.0e26f, 1.0e27f, 1.0e28f, 1.0e29f, 1.0e30f };
 
     @Override
     public void convertBoolean(BooleanPointable boolp, DataOutput dOut) throws SystemException, IOException {
@@ -121,14 +132,40 @@ public class CastToFloatOperation extend
                 decimalPlace += (negativeOffset ? -moveOffset : moveOffset);
             }
 
+            /*
+             * The following conditions to create the floating point value is using known valid float values.
+             * In addition, each one only needs one or two operations to get the float value, further minimizing
+             * possible errors. (Not perfect, but pretty good.)
+             */
             valueFloat = (float) value;
-            while (decimalPlace != 0 && valueFloat != 0) {
-                if (decimalPlace > 0) {
-                    --decimalPlace;
-                    valueFloat *= 10;
-                } else {
-                    ++decimalPlace;
-                    valueFloat /= 10;
+            if (decimalPlace == 0 || valueFloat == 0.0f) {
+                // No modification required to float value.
+            } else if (decimalPlace >= 0) {
+                if (decimalPlace <= 10) {
+                    valueFloat *= powersOf10upTo10[decimalPlace];
+                } else if (decimalPlace <= 20) {
+                    valueFloat *= powersOf10upTo10[10];
+                    valueFloat *= powersOf10upTo10[decimalPlace - 10];
+                } else if (decimalPlace <= 30) {
+                    valueFloat *= powersOf10from20to30[decimalPlace];
+                } else if (decimalPlace <= 38) {
+                    valueFloat *= powersOf10from20to30[30];
+                    valueFloat *= powersOf10upTo10[decimalPlace - 30];
+                }
+            } else {
+                if (decimalPlace >= -10) {
+                    valueFloat /= powersOf10upTo10[-decimalPlace];
+                } else if (decimalPlace >= -20) {
+                    valueFloat /= powersOf10upTo10[10];
+                    valueFloat /= powersOf10upTo10[-decimalPlace - 10];
+                } else if (decimalPlace >= -30) {
+                    valueFloat /= powersOf10from20to30[-decimalPlace];
+                } else if (decimalPlace >= -40) {
+                    valueFloat /= powersOf10from20to30[30];
+                    valueFloat /= powersOf10upTo10[-decimalPlace - 30];
+                } else if (decimalPlace >= -45) {
+                    valueFloat /= powersOf10from20to30[20];
+                    valueFloat /= powersOf10from20to30[-decimalPlace - 20];
                 }
             }
         }
@@ -142,4 +179,60 @@ public class CastToFloatOperation extend
         convertString(stringp, dOut);
     }
 
+    /**
+     * Derived Datatypes
+     */
+    public void convertByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(bytep, dOut);
+    }
+
+    public void convertInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(intp, dOut);
+    }
+
+    public void convertLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertNonNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertNonPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(shortp, dOut);
+    }
+
+    public void convertUnsignedByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(bytep, dOut);
+    }
+
+    public void convertUnsignedInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(intp, dOut);
+    }
+
+    public void convertUnsignedLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(longp, dOut);
+    }
+
+    public void convertUnsignedShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeDoubleValue(shortp, dOut);
+    }
+
+    private void writeDoubleValue(INumeric numericp, DataOutput dOut) throws SystemException, IOException {
+        dOut.write(ValueTag.XS_FLOAT_TAG);
+        dOut.writeFloat(numericp.floatValue());
+    }
+
 }
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,174 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.vxquery.datamodel.accessors.atomic.XSDecimalPointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.runtime.functions.strings.ICharacterIterator;
+import org.apache.vxquery.runtime.functions.strings.UTF8StringCharacterIterator;
+
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class CastToIntOperation extends AbstractCastToOperation {
+    boolean negativeAllowed = true;
+    int returnTag = ValueTag.XS_INT_TAG;
+
+    @Override
+    public void convertBoolean(BooleanPointable boolp, DataOutput dOut) throws SystemException, IOException {
+        dOut.write(returnTag);
+        dOut.writeInt((int) (boolp.getBoolean() ? 1 : 0));
+    }
+
+    @Override
+    public void convertDecimal(XSDecimalPointable decp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(decp, dOut);
+    }
+
+    @Override
+    public void convertDouble(DoublePointable doublep, DataOutput dOut) throws SystemException, IOException {
+        double doubleValue = doublep.getDouble();
+        if (Double.isInfinite(doubleValue) || Double.isNaN(doubleValue)) {
+            throw new SystemException(ErrorCode.FOCA0002);
+        }
+        if (doubleValue > Integer.MAX_VALUE || doubleValue < Integer.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FOCA0003);
+        }
+        if (doublep.byteValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        dOut.write(returnTag);
+        dOut.writeInt(doublep.intValue());
+    }
+
+    @Override
+    public void convertFloat(FloatPointable floatp, DataOutput dOut) throws SystemException, IOException {
+        float floatValue = floatp.getFloat();
+        if (Float.isInfinite(floatValue) || Float.isNaN(floatValue)) {
+            throw new SystemException(ErrorCode.FOCA0002);
+        }
+        if (floatValue > Integer.MAX_VALUE || floatValue < Integer.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FOCA0003);
+        }
+        if (floatp.byteValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        dOut.write(returnTag);
+        dOut.writeInt(floatp.intValue());
+    }
+
+    @Override
+    public void convertInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(longp, dOut);
+    }
+
+    @Override
+    public void convertString(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
+        ICharacterIterator charIterator = new UTF8StringCharacterIterator(stringp);
+        charIterator.reset();
+        long value = 0;
+        int c = 0;
+        boolean negative = false;
+
+        // Check the first character.
+        c = charIterator.next();
+        if (c == Character.valueOf('-') && negativeAllowed) {
+            negative = true;
+            c = charIterator.next();
+        }
+
+        // Read the numeric value.
+        do {
+            if (Character.isDigit(c)) {
+                value = value * 10 + Character.getNumericValue(c);
+            } else {
+                throw new SystemException(ErrorCode.FORG0001);
+            }
+        } while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR);
+
+        if (value > Integer.MAX_VALUE || value < Integer.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+
+        dOut.write(returnTag);
+        dOut.writeInt((int) (negative ? -value : value));
+    }
+
+    @Override
+    public void convertUntypedAtomic(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
+        convertString(stringp, dOut);
+    }
+
+    /**
+     * Derived Datatypes
+     */
+    public void convertByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(bytep, dOut);
+    }
+
+    public void convertInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(intp, dOut);
+    }
+
+    public void convertLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(longp, dOut);
+    }
+
+    public void convertNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(longp, dOut);
+    }
+
+    public void convertNonNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(longp, dOut);
+    }
+
+    public void convertNonPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(longp, dOut);
+    }
+
+    public void convertPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(longp, dOut);
+    }
+
+    public void convertShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(shortp, dOut);
+    }
+
+    public void convertUnsignedByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(bytep, dOut);
+    }
+
+    public void convertUnsignedInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(intp, dOut);
+    }
+
+    public void convertUnsignedLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(longp, dOut);
+    }
+
+    public void convertUnsignedShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeIntValue(shortp, dOut);
+    }
+
+    private void writeIntValue(INumeric numericp, DataOutput dOut) throws SystemException, IOException {
+        if (numericp.longValue() > Integer.MAX_VALUE || numericp.longValue() < Integer.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        if (numericp.intValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+
+        dOut.write(returnTag);
+        dOut.writeInt(numericp.intValue());
+    }
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntegerOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntegerOperation.java?rev=1371530&r1=1371529&r2=1371530&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntegerOperation.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToIntegerOperation.java Thu Aug  9 23:17:59 2012
@@ -10,24 +10,30 @@ import org.apache.vxquery.exceptions.Sys
 import org.apache.vxquery.runtime.functions.strings.ICharacterIterator;
 import org.apache.vxquery.runtime.functions.strings.UTF8StringCharacterIterator;
 
+import edu.uci.ics.hyracks.data.std.api.INumeric;
 import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class CastToIntegerOperation extends AbstractCastToOperation {
+    boolean negativeAllowed = true;
+    boolean negativeRequired = false;
+    int returnTag = ValueTag.XS_INTEGER_TAG;
 
     @Override
     public void convertBoolean(BooleanPointable boolp, DataOutput dOut) throws SystemException, IOException {
-        dOut.write(ValueTag.XS_INTEGER_TAG);
+        dOut.write(returnTag);
         dOut.writeLong((long) (boolp.getBoolean() ? 1 : 0));
     }
 
     @Override
     public void convertDecimal(XSDecimalPointable decp, DataOutput dOut) throws SystemException, IOException {
-        dOut.write(ValueTag.XS_INTEGER_TAG);
-        dOut.writeLong(decp.getBeforeDecimalPlace());
+        writeIntegerValue(decp, dOut);
     }
 
     @Override
@@ -39,7 +45,10 @@ public class CastToIntegerOperation exte
         if (doubleValue > Long.MAX_VALUE || doubleValue < Long.MIN_VALUE) {
             throw new SystemException(ErrorCode.FOCA0003);
         }
-        dOut.write(ValueTag.XS_INTEGER_TAG);
+        if ((doublep.longValue() < 0 && !negativeAllowed) || (doublep.longValue() > 0 && negativeRequired)) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        dOut.write(returnTag);
         dOut.writeLong(doublep.longValue());
     }
 
@@ -52,14 +61,16 @@ public class CastToIntegerOperation exte
         if (floatValue > Long.MAX_VALUE || floatValue < Long.MIN_VALUE) {
             throw new SystemException(ErrorCode.FOCA0003);
         }
-        dOut.write(ValueTag.XS_INTEGER_TAG);
+        if ((floatp.longValue() < 0 && !negativeAllowed) || (floatp.longValue() > 0 && negativeRequired)) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        dOut.write(returnTag);
         dOut.writeLong(floatp.longValue());
     }
 
     @Override
     public void convertInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
-        dOut.write(ValueTag.XS_INTEGER_TAG);
-        dOut.write(longp.getByteArray(), longp.getStartOffset(), longp.getLength());
+        writeIntegerValue(longp, dOut);
     }
 
     @Override
@@ -72,21 +83,26 @@ public class CastToIntegerOperation exte
 
         // Check the first character.
         c = charIterator.next();
-        if (c == Character.valueOf('-')) {
+        if (c == Character.valueOf('-') && negativeAllowed) {
             negative = true;
             c = charIterator.next();
+        } else if (negativeRequired) {
+            throw new SystemException(ErrorCode.FORG0001);
         }
 
         // Read the numeric value.
         do {
             if (Character.isDigit(c)) {
+                if (value > ((Long.MAX_VALUE - Character.getNumericValue(c)) / 10)) {
+                    throw new SystemException(ErrorCode.FOCA0001);
+                }
                 value = value * 10 + Character.getNumericValue(c);
             } else {
                 throw new SystemException(ErrorCode.FORG0001);
             }
         } while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR);
 
-        dOut.write(ValueTag.XS_INTEGER_TAG);
+        dOut.write(returnTag);
         dOut.writeLong((negative ? -value : value));
     }
 
@@ -95,4 +111,63 @@ public class CastToIntegerOperation exte
         convertString(stringp, dOut);
     }
 
+    /**
+     * Derived Datatypes
+     */
+    public void convertByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(bytep, dOut);
+    }
+
+    public void convertInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(intp, dOut);
+    }
+
+    public void convertLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(longp, dOut);
+    }
+
+    public void convertNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(longp, dOut);
+    }
+
+    public void convertNonNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(longp, dOut);
+    }
+
+    public void convertNonPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(longp, dOut);
+    }
+
+    public void convertPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(longp, dOut);
+    }
+
+    public void convertShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(shortp, dOut);
+    }
+
+    public void convertUnsignedByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(bytep, dOut);
+    }
+
+    public void convertUnsignedInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(intp, dOut);
+    }
+
+    public void convertUnsignedLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(longp, dOut);
+    }
+
+    public void convertUnsignedShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeIntegerValue(shortp, dOut);
+    }
+
+    private void writeIntegerValue(INumeric numericp, DataOutput dOut) throws SystemException, IOException {
+        if ((numericp.longValue() < 0 && !negativeAllowed) || (numericp.longValue() > 0 && negativeRequired)) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+
+        dOut.write(returnTag);
+        dOut.writeLong(numericp.longValue());
+    }
 }
\ No newline at end of file

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToLongOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToLongOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToLongOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToLongOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,8 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToLongOperation extends CastToIntegerOperation {
+    boolean negativeAllowed = true;
+    int returnTag = ValueTag.XS_LONG_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToLongOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNegativeIntegerOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNegativeIntegerOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNegativeIntegerOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNegativeIntegerOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,9 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToNegativeIntegerOperation extends CastToIntegerOperation {
+    boolean negativeAllowed = true;
+    boolean negativeRequired = true;
+    int returnTag = ValueTag.XS_NON_POSITIVE_INTEGER_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNegativeIntegerOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonNegativeIntegerOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonNegativeIntegerOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonNegativeIntegerOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonNegativeIntegerOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,9 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToNonNegativeIntegerOperation extends CastToIntegerOperation {
+    boolean negativeAllowed = true;
+    boolean negativeRequired = true;
+    int returnTag = ValueTag.XS_NON_POSITIVE_INTEGER_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonNegativeIntegerOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonPositiveIntegerOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonPositiveIntegerOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonPositiveIntegerOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonPositiveIntegerOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,8 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToNonPositiveIntegerOperation extends CastToIntegerOperation {
+    boolean negativeAllowed = false;
+    int returnTag = ValueTag.XS_NON_NEGATIVE_INTEGER_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToNonPositiveIntegerOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToPositiveIntegerOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToPositiveIntegerOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToPositiveIntegerOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToPositiveIntegerOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,8 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToPositiveIntegerOperation extends CastToIntegerOperation {
+    boolean negativeAllowed = false;
+    int returnTag = ValueTag.XS_POSITIVE_INTEGER_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToPositiveIntegerOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToShortOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToShortOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToShortOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToShortOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,174 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.vxquery.datamodel.accessors.atomic.XSDecimalPointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
+import org.apache.vxquery.exceptions.ErrorCode;
+import org.apache.vxquery.exceptions.SystemException;
+import org.apache.vxquery.runtime.functions.strings.ICharacterIterator;
+import org.apache.vxquery.runtime.functions.strings.UTF8StringCharacterIterator;
+
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.BytePointable;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class CastToShortOperation extends AbstractCastToOperation {
+    boolean negativeAllowed = true;
+    int returnTag = ValueTag.XS_SHORT_TAG;
+
+    @Override
+    public void convertBoolean(BooleanPointable boolp, DataOutput dOut) throws SystemException, IOException {
+        dOut.write(returnTag);
+        dOut.writeShort((short) (boolp.getBoolean() ? 1 : 0));
+    }
+
+    @Override
+    public void convertDecimal(XSDecimalPointable decp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(decp, dOut);
+    }
+
+    @Override
+    public void convertDouble(DoublePointable doublep, DataOutput dOut) throws SystemException, IOException {
+        double doubleValue = doublep.getDouble();
+        if (Double.isInfinite(doubleValue) || Double.isNaN(doubleValue)) {
+            throw new SystemException(ErrorCode.FOCA0002);
+        }
+        if (doubleValue > Short.MAX_VALUE || doubleValue < Short.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FOCA0003);
+        }
+        if (doublep.byteValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        dOut.write(returnTag);
+        dOut.writeShort(doublep.shortValue());
+    }
+
+    @Override
+    public void convertFloat(FloatPointable floatp, DataOutput dOut) throws SystemException, IOException {
+        float floatValue = floatp.getFloat();
+        if (Float.isInfinite(floatValue) || Float.isNaN(floatValue)) {
+            throw new SystemException(ErrorCode.FOCA0002);
+        }
+        if (floatValue > Short.MAX_VALUE || floatValue < Short.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FOCA0003);
+        }
+        if (floatp.byteValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        dOut.write(returnTag);
+        dOut.writeShort(floatp.shortValue());
+    }
+
+    @Override
+    public void convertInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(longp, dOut);
+    }
+
+    @Override
+    public void convertString(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
+        ICharacterIterator charIterator = new UTF8StringCharacterIterator(stringp);
+        charIterator.reset();
+        long value = 0;
+        int c = 0;
+        boolean negative = false;
+
+        // Check the first character.
+        c = charIterator.next();
+        if (c == Character.valueOf('-') && negativeAllowed) {
+            negative = true;
+            c = charIterator.next();
+        }
+
+        // Read the numeric value.
+        do {
+            if (Character.isDigit(c)) {
+                value = value * 10 + Character.getNumericValue(c);
+            } else {
+                throw new SystemException(ErrorCode.FORG0001);
+            }
+        } while ((c = charIterator.next()) != ICharacterIterator.EOS_CHAR);
+
+        if (value > Short.MAX_VALUE || value < Short.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+
+        dOut.write(returnTag);
+        dOut.writeShort((short) (negative ? -value : value));
+    }
+
+    @Override
+    public void convertUntypedAtomic(UTF8StringPointable stringp, DataOutput dOut) throws SystemException, IOException {
+        convertString(stringp, dOut);
+    }
+
+    /**
+     * Derived Datatypes
+     */
+    public void convertByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(bytep, dOut);
+    }
+
+    public void convertInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(intp, dOut);
+    }
+
+    public void convertLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(longp, dOut);
+    }
+
+    public void convertNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(longp, dOut);
+    }
+
+    public void convertNonNegativeInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(longp, dOut);
+    }
+
+    public void convertNonPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(longp, dOut);
+    }
+
+    public void convertPositiveInteger(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(longp, dOut);
+    }
+
+    public void convertShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(shortp, dOut);
+    }
+
+    public void convertUnsignedByte(BytePointable bytep, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(bytep, dOut);
+    }
+
+    public void convertUnsignedInt(IntegerPointable intp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(intp, dOut);
+    }
+
+    public void convertUnsignedLong(LongPointable longp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(longp, dOut);
+    }
+
+    public void convertUnsignedShort(ShortPointable shortp, DataOutput dOut) throws SystemException, IOException {
+        writeShortValue(shortp, dOut);
+    }
+
+    private void writeShortValue(INumeric numericp, DataOutput dOut) throws SystemException, IOException {
+        if (numericp.longValue() > Short.MAX_VALUE || numericp.longValue() < Short.MIN_VALUE) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+        if (numericp.shortValue() < 0 && !negativeAllowed) {
+            throw new SystemException(ErrorCode.FORG0001);
+        }
+
+        dOut.write(returnTag);
+        dOut.writeShort(numericp.shortValue());
+    }
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToShortOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java?rev=1371530&r1=1371529&r2=1371530&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToStringOperation.java Thu Aug  9 23:17:59 2012
@@ -102,7 +102,7 @@ public class CastToStringOperation exten
             long pow10 = (long) Math.pow(10, nDigits - 1);
             int start = Math.max(decimalPlace, nDigits - 1);
             int end = Math.min(0, decimalPlace);
-
+            
             for (int i = start; i >= end; --i) {
                 if (i >= nDigits || i < 0) {
                     writeChar('0', dOutInner);
@@ -136,7 +136,7 @@ public class CastToStringOperation exten
         }
     }
 
-    private void convertDoubleCanonical(DoublePointable doublep, DataOutput dOut) throws SystemException, IOException {
+    public void convertDoubleCanonical(DoublePointable doublep, DataOutput dOut) throws SystemException, IOException {
         abvsInner.reset();
         double value = doublep.getDouble();
 
@@ -171,7 +171,7 @@ public class CastToStringOperation exten
                 writeCharSequence("0.0", dOutInner);
             } else {
                 for (int i = nDigits - 1; i >= 0; --i) {
-                    writeChar((char) ('0' + (value / pow10)), dOutInner);
+                    writeChar((char) ('0' + Math.floor(value / pow10)), dOutInner);
                     value %= pow10;
                     pow10 /= 10;
                     if (i == nDigits - 1) {
@@ -318,11 +318,11 @@ public class CastToStringOperation exten
             decp.set(abvsInner.getByteArray(), abvsInner.getStartOffset() + 1, abvsInner.getLength());
             convertDecimal(decp, dOut);
         } else {
-            convertFloatCononical(floatp, dOut);
+            convertFloatCanonical(floatp, dOut);
         }
     }
 
-    private void convertFloatCononical(FloatPointable floatp, DataOutput dOut) throws SystemException, IOException {
+    public void convertFloatCanonical(FloatPointable floatp, DataOutput dOut) throws SystemException, IOException {
         abvsInner.reset();
         float value = floatp.getFloat();
 
@@ -357,7 +357,7 @@ public class CastToStringOperation exten
                 writeCharSequence("0.0", dOutInner);
             } else {
                 for (int i = nDigits - 1; i >= 0; --i) {
-                    writeChar((char) ('0' + (value / pow10)), dOutInner);
+                    writeChar((char) ('0' + Math.floor(value / pow10)), dOutInner);
                     value %= pow10;
                     pow10 /= 10;
                     if (i == nDigits - 1) {
@@ -577,7 +577,7 @@ public class CastToStringOperation exten
      * @param value
      * @return
      */
-    private boolean isNumberPostive(long value) {
+    public static boolean isNumberPostive(long value) {
         return ((value & 0x8000000000000000L) == 0 ? true : false);
     }
 

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedByteOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedByteOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedByteOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedByteOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,8 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToUnsignedByteOperation extends CastToByteOperation {
+    boolean negativeAllowed = false;
+    int returnTag = ValueTag.XS_UNSIGNED_BYTE_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedByteOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedIntOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedIntOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedIntOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedIntOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,8 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToUnsignedIntOperation extends CastToIntOperation {
+    boolean negativeAllowed = false;
+    int returnTag = ValueTag.XS_UNSIGNED_INT_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedIntOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedLongOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedLongOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedLongOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedLongOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,8 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToUnsignedLongOperation extends CastToIntegerOperation {
+    boolean negativeAllowed = false;
+    int returnTag = ValueTag.XS_UNSIGNED_LONG_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedLongOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedShortOperation.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedShortOperation.java?rev=1371530&view=auto
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedShortOperation.java (added)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedShortOperation.java Thu Aug  9 23:17:59 2012
@@ -0,0 +1,8 @@
+package org.apache.vxquery.runtime.functions.cast;
+
+import org.apache.vxquery.datamodel.values.ValueTag;
+
+public class CastToUnsignedShortOperation extends CastToShortOperation {
+    boolean negativeAllowed = false;
+    int returnTag = ValueTag.XS_UNSIGNED_SHORT_TAG;
+}
\ No newline at end of file

Propchange: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/cast/CastToUnsignedShortOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native