You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/01/28 21:40:23 UTC

[GitHub] [arrow] suchagit opened a new pull request #9356: [ARROW-11422] [C#] add decimal support

suchagit opened a new pull request #9356:
URL: https://github.com/apache/arrow/pull/9356


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#issuecomment-769419041


   https://issues.apache.org/jira/browse/ARROW-11422


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r568754230



##########
File path: csharp/src/Apache.Arrow/Arrays/DecimalArray.cs
##########
@@ -0,0 +1,45 @@
+// 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.
+
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DecimalArray : PrimitiveArray<decimal>

Review comment:
       I don't think the C# in-memory format of decimal is the same layout as the Arrow decimal format, is it? I'm trying to find the documentation on the memory layout, the only thing I can find is in the Schema.fbs:
   
   https://github.com/apache/arrow/blob/e676aeacb2c6597c063faeed20d6142b0df938d8/format/Schema.fbs#L176-L185
   
   The C# decimal layout is different:
   
   https://docs.microsoft.com/en-us/dotnet/api/system.decimal?view=net-5.0#remarks
   
   > The binary representation of a Decimal value consists of a 1-bit sign, a 96-bit integer number, and a scaling factor used to divide the 96-bit integer and specify what portion of it is a decimal fraction. The scaling factor is implicitly the number 10, raised to an exponent ranging from 0 to 28.

##########
File path: csharp/src/Apache.Arrow/Types/DecimalType.cs
##########
@@ -17,6 +17,7 @@ namespace Apache.Arrow.Types
 {
     public sealed class DecimalType: FixedSizeBinaryType
     {
+        public static readonly DecimalType Default = new DecimalType(0, 0);

Review comment:
       I'm not sure a decimal with precision = 0 is valid.
   
   Looking at the C++ implementation, I don't see a "default" DecimalType being defined. Maybe it is best to leave it up to the caller?

##########
File path: csharp/src/Apache.Arrow/Arrays/DecimalArray.cs
##########
@@ -0,0 +1,45 @@
+// 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.
+
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DecimalArray : PrimitiveArray<decimal>

Review comment:
       Thanks @emkornfield. One thing that I don't quite understand is the `precision` field in metadata. The data is fixed sized right? So each value will always take up 128 or 256 bits. But if I set the precision lower than `38` for a Decimal128, what exactly does that mean? Does it mean to ignore digits at the beginning of the integer? For example:
   
   ```
   Precision = 3
   Scale = 0
   Data Value = 0b010111011100   (1,500 in binary)
   ```
   
   I assume this number means `500`, and the leading `1,000` gets truncated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] suchagit commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
suchagit commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r585730134



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+            {
+                bytes = integerBytes;
+                return;
+            }
+
+            if (integerValue.Sign == -1)
+            {
+                integerBytes.CopyTo(bytes);
+                for (int i = integerBytes.Length; i < byteWidth; i++)
+                {
+                    bytes[i] = 255;
+                }
+            }
+            else
+            {
+                integerBytes.CopyTo(bytes);
+            }
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());

Review comment:
       I've made another round of changes based on the feedback.
   These mostly combine CheckPrecisionAndScale and GetBytes - as you mention they need to be called together, and make some of the other suggestions / remove code which is not needed. [I have removed the DecimalLayout struct and tried to use netcore features where possibl - the exception being using the above for decimal.GetBits as a Span<int> which is only available on netcore 5.0 from what I can tell].
   I've also added a few tests on the decimal utility, which is resulted in me making this public again, and also updated the docs.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt closed pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt closed pull request #9356:
URL: https://github.com/apache/arrow/pull/9356


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r568962099



##########
File path: csharp/src/Apache.Arrow/Arrays/DecimalArray.cs
##########
@@ -0,0 +1,45 @@
+// 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.
+
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DecimalArray : PrimitiveArray<decimal>

Review comment:
       The documentation is trying to say that the Decimal value is either a 128-bit or a 256-bit twos complement integer.  The metadata of the type determines the scaling factor of the integer.  So yes, it sounds like they are quite different.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] suchagit commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
suchagit commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r591750861



##########
File path: csharp/test/Apache.Arrow.Tests/DecimalUtilityTests.cs
##########
@@ -0,0 +1,49 @@
+// 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.
+
+using System;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class DecimalUtilityTests
+    {
+        public class Overflow
+        {
+            [Theory]
+            [InlineData(100.123, 10, 4, false)]
+            [InlineData(100.123, 6, 4, false)]
+            [InlineData(100.123, 3, 3, true)]
+            [InlineData(100.123, 10, 2, true)]
+            [InlineData(100.123, 5, 2, true)]
+            [InlineData(100.123, 5, 3, true)]
+            [InlineData(100.123, 6, 3, false)]
+            public void HasExpectedResultOrThrows(decimal d, int precision , int scale, bool shouldThrow)
+            {
+                var builder = new Decimal128Array.Builder(new Decimal128Type(precision, scale));
+
+                if (shouldThrow)
+                {
+                   Assert.Throws<OverflowException>(() => builder.Append(d));
+                }
+                else
+                {
+                    builder.Append(d);

Review comment:
       Sure, that's done now




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #9356: [ARROW-11422] [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#issuecomment-769417353


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#issuecomment-771344773


   I'm not familiar with the C# implementation but note decimals can now be 128 or 256 bit.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] suchagit commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
suchagit commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r591731025



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,144 @@
+// 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.
+
+using System;
+using System.Numerics;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility

Review comment:
       I have had some time to look at this and the other suggestions and pushed some changes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] suchagit commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
suchagit commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r585730554



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+            {
+                bytes = integerBytes;
+                return;
+            }
+
+            if (integerValue.Sign == -1)
+            {
+                integerBytes.CopyTo(bytes);
+                for (int i = integerBytes.Length; i < byteWidth; i++)
+                {
+                    bytes[i] = 255;
+                }
+            }
+            else
+            {
+                integerBytes.CopyTo(bytes);
+            }
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());
+
+            if (layout.Scale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + layout.Scale + " != " + scale);

Review comment:
       Added

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+            {
+                bytes = integerBytes;
+                return;
+            }
+
+            if (integerValue.Sign == -1)
+            {
+                integerBytes.CopyTo(bytes);
+                for (int i = integerBytes.Length; i < byteWidth; i++)
+                {
+                    bytes[i] = 255;
+                }
+            }
+            else
+            {
+                integerBytes.CopyTo(bytes);
+            }
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());
+
+            if (layout.Scale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + layout.Scale + " != " + scale);
+
+            if(integerValue >= BigInteger.Pow(10, precision))
+                throw new OverflowException("Decimal precision can not be greater than that in the Arrow vector: " + value + " has precision > " + precision);
+
+            if (layout.Scale < scale) // pad with trailing zeros
+            {
+                integerValue *= BigInteger.Pow(10, scale - layout.Scale);
+            }
+
+            if (value < 0) // sign the big int
+                integerValue = -integerValue;
+
+            return true;
+        }
+
+        [StructLayout(LayoutKind.Explicit)]
+        private readonly struct DecimalLayout
+        {
+            public DecimalLayout(decimal value)
+            {
+                this = default;
+                d = value;
+            }
+
+            [FieldOffset(0)] private readonly decimal d;
+
+            [FieldOffset(0)] private readonly int flags;
+            [FieldOffset(4)] private readonly int hi;
+#if BIGENDIAN

Review comment:
       I've removed this as part of the latest refactor




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r584834740



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+            {
+                bytes = integerBytes;
+                return;
+            }
+
+            if (integerValue.Sign == -1)
+            {
+                integerBytes.CopyTo(bytes);
+                for (int i = integerBytes.Length; i < byteWidth; i++)
+                {
+                    bytes[i] = 255;
+                }
+            }
+            else
+            {
+                integerBytes.CopyTo(bytes);
+            }
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());

Review comment:
       This comment and code doesn't really make sense together. The comment says it is using `DecimalLayout` instead of `decimal.GetBits(value)` to avoid an allocation. But then the next line allocates 3 separate byte arrays, and then 2 Concat enumerables, and then a final byte array to put them all together. You might as well have just called `decimal.GetBits(value)`.
   
   Also - on .NET Core, you can call https://docs.microsoft.com/en-us/dotnet/api/system.decimal.getbits?view=net-5.0#System_Decimal_GetBits_System_Decimal_System_Span_System_Int32__, which takes a `Span<int>`.
   
   Also, one last thought - it may make sense to combine `DecimalUtility.CheckPrecisionAndScale` and `GetBytes` into one method. They are only ever called together, and that way you don't necessarily need to go through an in-between structure if not needed.

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();

Review comment:
       Instead of `ToByteArray()`, on .NET Core you can use `TryWriteBytes`, which will write directly to the `Span<byte> bytes`. This will save an allocation and a copy.
   
   See https://docs.microsoft.com/en-us/dotnet/api/system.numerics.biginteger.trywritebytes?view=net-5.0

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+            {
+                bytes = integerBytes;
+                return;
+            }
+
+            if (integerValue.Sign == -1)
+            {
+                integerBytes.CopyTo(bytes);
+                for (int i = integerBytes.Length; i < byteWidth; i++)
+                {
+                    bytes[i] = 255;
+                }
+            }
+            else
+            {
+                integerBytes.CopyTo(bytes);
+            }
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());
+
+            if (layout.Scale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + layout.Scale + " != " + scale);
+
+            if(integerValue >= BigInteger.Pow(10, precision))
+                throw new OverflowException("Decimal precision can not be greater than that in the Arrow vector: " + value + " has precision > " + precision);
+
+            if (layout.Scale < scale) // pad with trailing zeros
+            {
+                integerValue *= BigInteger.Pow(10, scale - layout.Scale);
+            }
+
+            if (value < 0) // sign the big int
+                integerValue = -integerValue;
+
+            return true;
+        }
+
+        [StructLayout(LayoutKind.Explicit)]
+        private readonly struct DecimalLayout
+        {
+            public DecimalLayout(decimal value)
+            {
+                this = default;
+                d = value;
+            }
+
+            [FieldOffset(0)] private readonly decimal d;
+
+            [FieldOffset(0)] private readonly int flags;
+            [FieldOffset(4)] private readonly int hi;
+#if BIGENDIAN

Review comment:
       This isn't going to work well because no one builds this assembly with `BIGENDIAN` enabled.
   
   How about using:
   
   ```C#
           private readonly int _flags;
           private readonly uint _hi32;
           private readonly ulong _lo64;
   ```
   
   Which is what System.Decimal uses?
   
   To get the Low and the Mid, you can do:
   
   ```C#
   internal uint Low => (uint)_lo64;
   internal uint Mid => (uint)(_lo64 >> 32);
   ```

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)

Review comment:
       ```suggestion
           public static void GetBytes(BigInteger integerValue, int byteWidth, Span<byte> bytes)
   ```
   
   There isn't a need to use `ref` here.

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+            {
+                bytes = integerBytes;
+                return;
+            }
+
+            if (integerValue.Sign == -1)
+            {
+                integerBytes.CopyTo(bytes);
+                for (int i = integerBytes.Length; i < byteWidth; i++)
+                {
+                    bytes[i] = 255;
+                }
+            }
+            else
+            {
+                integerBytes.CopyTo(bytes);
+            }
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());
+
+            if (layout.Scale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + layout.Scale + " != " + scale);
+
+            if(integerValue >= BigInteger.Pow(10, precision))
+                throw new OverflowException("Decimal precision can not be greater than that in the Arrow vector: " + value + " has precision > " + precision);
+
+            if (layout.Scale < scale) // pad with trailing zeros
+            {
+                integerValue *= BigInteger.Pow(10, scale - layout.Scale);
+            }
+
+            if (value < 0) // sign the big int
+                integerValue = -integerValue;
+
+            return true;
+        }
+
+        [StructLayout(LayoutKind.Explicit)]
+        private readonly struct DecimalLayout
+        {
+            public DecimalLayout(decimal value)
+            {
+                this = default;

Review comment:
       Why set `this = default`, when we set `d = value` right afterwards?

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+            {
+                bytes = integerBytes;
+                return;
+            }
+
+            if (integerValue.Sign == -1)
+            {
+                integerBytes.CopyTo(bytes);
+                for (int i = integerBytes.Length; i < byteWidth; i++)
+                {
+                    bytes[i] = 255;
+                }
+            }
+            else
+            {
+                integerBytes.CopyTo(bytes);
+            }
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());
+
+            if (layout.Scale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + layout.Scale + " != " + scale);

Review comment:
       Do we have tests for these overflow scenarios?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] suchagit commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
suchagit commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r580836951



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,188 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility
+    {
+        private static readonly byte[] _minusOne = { 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255 };
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static byte[] GetBytes(BigInteger integerValue, int byteWidth)
+        {
+            byte[] integerBytes = integerValue.ToByteArray();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+                return integerBytes;
+
+            byte[] result = new byte[byteWidth];
+            if (integerValue.Sign == -1)
+            {
+                Buffer.BlockCopy(integerBytes, 0, result, 0, integerBytes.Length);
+                Buffer.BlockCopy(_minusOne, 0, result, integerBytes.Length, byteWidth - integerBytes.Length);
+            } else
+            {
+                Buffer.BlockCopy(integerBytes, 0, result, 0, integerBytes.Length);
+            }
+
+            return result;
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());
+
+            if (layout.Scale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + layout.Scale + " != " + scale);
+
+            if(integerValue >= BigInteger.Pow(10, precision))
+                throw new OverflowException("Decimal precision can not be greater than that in the Arrow vector: " + value + " has precision > " + precision);
+
+            if (layout.Scale < scale) // pad with trailing zeros
+            {
+                integerValue *= BigInteger.Pow(10, scale - layout.Scale);
+            }
+
+            if (value < 0) // sign the big int
+                integerValue = -integerValue;
+
+            return true;
+        }
+
+        private static decimal GetDecimalFromBigInteger(BigInteger value, int scale)
+        {
+            var b = new BigIntegerLayout(value);
+            if (b.Bits == null)
+                return b.Sign;
+
+            int length = b.Bits.Length;
+            if (length > 3) throw new OverflowException("Decimal overflow");
+
+            int lo = 0, mi = 0, hi = 0;
+
+            unchecked
+            {
+                if (length > 2) hi = b.Bits[2];
+                if (length > 1) mi = b.Bits[1];
+                if (length > 0) lo = b.Bits[0];
+            }
+
+            return new decimal(lo, mi, hi, b.Sign < 0, (byte)scale);
+        }
+
+        [StructLayout(LayoutKind.Explicit)]
+        readonly struct BigIntegerLayout
+        {
+            public BigIntegerLayout(BigInteger value)
+            {
+                this = default;
+                bi = value;
+            }
+
+            [FieldOffset(0)] readonly BigInteger bi;
+            [FieldOffset(0)] readonly int[] bits;
+
+            public int Sign => bi.Sign;
+            public int[] Bits =>  bits;
+        }
+
+
+        [StructLayout(LayoutKind.Explicit)]
+        readonly struct DecimalLayout
+        {
+            public DecimalLayout(decimal value)
+            {
+                this = default;
+                d = value;
+            }
+
+            [FieldOffset(0)] readonly decimal d;
+            [FieldOffset(0)] readonly int flags;
+            [FieldOffset(4)] readonly int hi;
+            [FieldOffset(8)] readonly int lo;
+            [FieldOffset(12)] readonly int mid;

Review comment:
       Updated to support both big and little endian systems, accessing lo / mid.
   Agreed it doesn't match exactly how Decimal is defined but I have since found that this is how DecCalc accesses: https://github.com/dotnet/runtime/blob/master/src/libraries/System.Private.CoreLib/src/System/Decimal.DecCalc.cs#L40




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r568962099



##########
File path: csharp/src/Apache.Arrow/Arrays/DecimalArray.cs
##########
@@ -0,0 +1,45 @@
+// 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.
+
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DecimalArray : PrimitiveArray<decimal>

Review comment:
       The documentation is trying to say that the Decimal value is either a 128-bit or a 256-bit twos complement integer.  The metadata of the type determines the scaling factor of the integer.  So yes, it sounds like they are quite different.

##########
File path: csharp/src/Apache.Arrow/Arrays/DecimalArray.cs
##########
@@ -0,0 +1,45 @@
+// 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.
+
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DecimalArray : PrimitiveArray<decimal>

Review comment:
       Yes, we ignore most significant bits of the integer (that would correspond to higher necessary precision).  I'm sure some code might break if those bits are populated.  But for instance we use this fact when writing to parquet: https://github.com/apache/arrow/blob/0e8de085f4afd3c9aa9c9ed05405a6c6ef8c360a/cpp/src/parquet/column_writer.cc




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r580440107



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,188 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility
+    {
+        private static readonly byte[] _minusOne = { 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255 };
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static byte[] GetBytes(BigInteger integerValue, int byteWidth)
+        {
+            byte[] integerBytes = integerValue.ToByteArray();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+                return integerBytes;
+
+            byte[] result = new byte[byteWidth];
+            if (integerValue.Sign == -1)
+            {
+                Buffer.BlockCopy(integerBytes, 0, result, 0, integerBytes.Length);
+                Buffer.BlockCopy(_minusOne, 0, result, integerBytes.Length, byteWidth - integerBytes.Length);
+            } else
+            {
+                Buffer.BlockCopy(integerBytes, 0, result, 0, integerBytes.Length);
+            }
+
+            return result;
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());
+
+            if (layout.Scale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + layout.Scale + " != " + scale);
+
+            if(integerValue >= BigInteger.Pow(10, precision))
+                throw new OverflowException("Decimal precision can not be greater than that in the Arrow vector: " + value + " has precision > " + precision);
+
+            if (layout.Scale < scale) // pad with trailing zeros
+            {
+                integerValue *= BigInteger.Pow(10, scale - layout.Scale);
+            }
+
+            if (value < 0) // sign the big int
+                integerValue = -integerValue;
+
+            return true;
+        }
+
+        private static decimal GetDecimalFromBigInteger(BigInteger value, int scale)
+        {
+            var b = new BigIntegerLayout(value);
+            if (b.Bits == null)
+                return b.Sign;
+
+            int length = b.Bits.Length;
+            if (length > 3) throw new OverflowException("Decimal overflow");
+
+            int lo = 0, mi = 0, hi = 0;
+
+            unchecked
+            {
+                if (length > 2) hi = b.Bits[2];
+                if (length > 1) mi = b.Bits[1];
+                if (length > 0) lo = b.Bits[0];
+            }
+
+            return new decimal(lo, mi, hi, b.Sign < 0, (byte)scale);
+        }
+
+        [StructLayout(LayoutKind.Explicit)]
+        readonly struct BigIntegerLayout
+        {
+            public BigIntegerLayout(BigInteger value)
+            {
+                this = default;
+                bi = value;
+            }
+
+            [FieldOffset(0)] readonly BigInteger bi;
+            [FieldOffset(0)] readonly int[] bits;
+
+            public int Sign => bi.Sign;
+            public int[] Bits =>  bits;
+        }
+
+
+        [StructLayout(LayoutKind.Explicit)]
+        readonly struct DecimalLayout
+        {
+            public DecimalLayout(decimal value)
+            {
+                this = default;
+                d = value;
+            }
+
+            [FieldOffset(0)] readonly decimal d;
+            [FieldOffset(0)] readonly int flags;
+            [FieldOffset(4)] readonly int hi;
+            [FieldOffset(8)] readonly int lo;
+            [FieldOffset(12)] readonly int mid;

Review comment:
       This isn't exactly how System.Decimal is defined. Would this work on big endian systems?
   
   https://github.com/dotnet/runtime/blob/8a52f1e948b6f22f418817ec1068f07b8dae2aa5/src/libraries/System.Private.CoreLib/src/System/Decimal.cs#L106-L108
   
   ```
           private readonly int _flags;
           private readonly uint _hi32;
           private readonly ulong _lo64;
   ```
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] suchagit commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
suchagit commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r585731207



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,163 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    internal static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(BigInteger integerValue, int byteWidth, ref Span<byte> bytes)
+        {
+            if (bytes.Length != byteWidth)
+            {
+                throw new OverflowException("ValueBuffer size not equal to " + byteWidth + " byte width: " + bytes.Length);
+            }
+
+            Span<byte> integerBytes = integerValue.ToByteArray().AsSpan();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+            {
+                bytes = integerBytes;
+                return;
+            }
+
+            if (integerValue.Sign == -1)
+            {
+                integerBytes.CopyTo(bytes);
+                for (int i = integerBytes.Length; i < byteWidth; i++)
+                {
+                    bytes[i] = 255;
+                }
+            }
+            else
+            {
+                integerBytes.CopyTo(bytes);
+            }
+        }
+
+        public static bool CheckPrecisionAndScale(decimal value, int precision, int scale, out BigInteger integerValue)
+        {
+            DecimalLayout layout = new DecimalLayout(value); // use in place of decimal.GetBits(value) to avoid an allocation
+            integerValue = new BigInteger(BitConverter.GetBytes(layout.Lo).Concat(BitConverter.GetBytes(layout.Mid)).Concat(BitConverter.GetBytes(layout.Hi)).ToArray());
+
+            if (layout.Scale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + layout.Scale + " != " + scale);
+
+            if(integerValue >= BigInteger.Pow(10, precision))
+                throw new OverflowException("Decimal precision can not be greater than that in the Arrow vector: " + value + " has precision > " + precision);
+
+            if (layout.Scale < scale) // pad with trailing zeros
+            {
+                integerValue *= BigInteger.Pow(10, scale - layout.Scale);
+            }
+
+            if (value < 0) // sign the big int
+                integerValue = -integerValue;
+
+            return true;
+        }
+
+        [StructLayout(LayoutKind.Explicit)]
+        private readonly struct DecimalLayout
+        {
+            public DecimalLayout(decimal value)
+            {
+                this = default;

Review comment:
       I've removed this as part of the latest refactor (I think this was needed though)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r579334666



##########
File path: csharp/src/Apache.Arrow/Flatbuf/BodyCompressionMethod.cs
##########
@@ -0,0 +1,24 @@
+// <auto-generated>
+//  automatically generated by the FlatBuffers compiler, do not modify
+// </auto-generated>
+
+namespace Apache.Arrow.Flatbuf
+{
+
+/// Provided for forward compatibility in case we need to support different
+/// strategies for compressing the IPC message body (like whole-body
+/// compression rather than buffer-level) in the future
+public enum BodyCompressionMethod : sbyte

Review comment:
       These enums shouldn't be `public`. Nothing in `Flatbuf` should be public.

##########
File path: csharp/test/Apache.Arrow.Tests/ArrowReaderVerifier.cs
##########
@@ -147,6 +150,40 @@ private void CompareBinaryArrays<T>(BinaryArray actualArray)
                 Assert.True(expectedArray.Values.Slice(0, expectedArray.Length).SequenceEqual(actualArray.Values.Slice(0, actualArray.Length)));
             }
 
+            private void CompareArrays(Decimal128Array actualArray)

Review comment:
       Can we combine these two methods into one for a `FixedSizeBinaryArray`?

##########
File path: csharp/src/Apache.Arrow/ArrowBuffer.Builder.cs
##########
@@ -211,6 +211,23 @@ public ArrowBuffer Build(MemoryAllocator allocator = default)
                 return new ArrowBuffer(memoryOwner);
             }
 
+            /// <summary>
+            /// Build an Arrow buffer from the appended contents so far of the specified byte size.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an <see cref="ArrowBuffer"/> object.</returns>
+            public ArrowBuffer Build(int byteSize, MemoryAllocator allocator = default)

Review comment:
       Does this need to be public? Are we expecting external callers to use it?

##########
File path: csharp/test/Apache.Arrow.Tests/TestData.cs
##########
@@ -48,8 +48,9 @@ public static RecordBatch CreateSampleRecordBatch(int length, int columnSetCount
                 builder.Field(CreateField(TimestampType.Default, i));
                 builder.Field(CreateField(StringType.Default, i));
                 builder.Field(CreateField(new StructType(new List<Field> { CreateField(StringType.Default, i), CreateField(Int32Type.Default, i) }), i));
+                builder.Field(CreateField(new Decimal128Type(20, 10), i));
+                builder.Field(CreateField(new Decimal256Type(20, 10), i));

Review comment:
       Can we test with different precision and scales?

##########
File path: csharp/test/Apache.Arrow.Benchmarks/ArrowReaderBenchmark.cs
##########
@@ -116,6 +116,10 @@ private static double SumAllNumbers(RecordBatch recordBatch)
                         DoubleArray doubleArray = (DoubleArray)array;
                         sum += Sum(doubleArray);
                         break;
+                    case ArrowTypeId.Decimal128:
+                        Decimal128Array decimalArray = (Decimal128Array)array;
+                        sum += (double)Sum(decimalArray);

Review comment:
       This cast is unnecessary since the `Sum` method already returns a `double`.

##########
File path: csharp/src/Apache.Arrow/ArrowBuffer.Builder.cs
##########
@@ -211,6 +211,23 @@ public ArrowBuffer Build(MemoryAllocator allocator = default)
                 return new ArrowBuffer(memoryOwner);
             }
 
+            /// <summary>
+            /// Build an Arrow buffer from the appended contents so far of the specified byte size.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an <see cref="ArrowBuffer"/> object.</returns>
+            public ArrowBuffer Build(int byteSize, MemoryAllocator allocator = default)
+            {
+                int currentBytesLength = Length * _size;

Review comment:
       Can we refactor the above method to call into this method with `64`? That way there is less duplicated code.

##########
File path: csharp/src/Apache.Arrow/Arrays/Decimal128Array.cs
##########
@@ -0,0 +1,96 @@
+// 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.
+
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Numerics;
+using Apache.Arrow.Arrays;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class Decimal128Array : FixedSizeBinaryArray
+    {
+        public class Builder : BuilderBase<Decimal128Array, Builder>
+        {
+            public Builder(Decimal128Type type) : base(type, 16)
+            {
+                DataType = type;
+            }
+
+            protected new Decimal128Type DataType { get; }
+
+            protected override Decimal128Array Build(ArrayData data)
+            {
+                return new Decimal128Array(data);
+            }
+
+            public Builder Append(decimal value)
+            {
+                DecimalUtility.CheckPrecisionAndScale(value, DataType.Precision, DataType.Scale, out BigInteger integerValue);
+                byte[] bytes = DecimalUtility.GetBytes(integerValue, DataType.ByteWidth);
+
+                return Append(bytes);
+            }
+
+            public Builder AppendRange(IEnumerable<decimal> values)
+            {
+                if (values == null)
+                {
+                    throw new ArgumentNullException(nameof(values));
+                }
+
+                foreach (decimal d in values)
+                {
+                    Append(d);
+                }
+
+                return Instance;
+            }
+
+            public Builder Set(int index, decimal value)
+            {
+                DecimalUtility.CheckPrecisionAndScale(value, DataType.Precision, DataType.Scale, out BigInteger integerValue);
+                byte[] bytes = DecimalUtility.GetBytes(integerValue, DataType.ByteWidth);

Review comment:
       A different way that doesn't involve `stackalloc` would be to get a `Span<byte>` directly into the `ValueBuffer` itself (after making sure it is long enough) and passing that to `DecimalUtility.GetBytes`.

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,188 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility

Review comment:
       Does this need to be public?

##########
File path: csharp/src/Apache.Arrow/Arrays/Decimal128Array.cs
##########
@@ -0,0 +1,96 @@
+// 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.
+
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Numerics;
+using Apache.Arrow.Arrays;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class Decimal128Array : FixedSizeBinaryArray
+    {
+        public class Builder : BuilderBase<Decimal128Array, Builder>
+        {
+            public Builder(Decimal128Type type) : base(type, 16)
+            {
+                DataType = type;
+            }
+
+            protected new Decimal128Type DataType { get; }
+
+            protected override Decimal128Array Build(ArrayData data)
+            {
+                return new Decimal128Array(data);
+            }
+
+            public Builder Append(decimal value)
+            {
+                DecimalUtility.CheckPrecisionAndScale(value, DataType.Precision, DataType.Scale, out BigInteger integerValue);
+                byte[] bytes = DecimalUtility.GetBytes(integerValue, DataType.ByteWidth);
+
+                return Append(bytes);
+            }
+
+            public Builder AppendRange(IEnumerable<decimal> values)
+            {
+                if (values == null)
+                {
+                    throw new ArgumentNullException(nameof(values));
+                }
+
+                foreach (decimal d in values)
+                {
+                    Append(d);
+                }
+
+                return Instance;
+            }
+
+            public Builder Set(int index, decimal value)
+            {
+                DecimalUtility.CheckPrecisionAndScale(value, DataType.Precision, DataType.Scale, out BigInteger integerValue);
+                byte[] bytes = DecimalUtility.GetBytes(integerValue, DataType.ByteWidth);

Review comment:
       This is a bit of a performance trap, since it creates a new `byte[]` every time it is called.
   
   Instead, we should change this around so this method `stackalloc`s a buffer for `DecimalUtility.GetBytes` to write into. It can then be passed as a `Span<byte>` of length ByteWidth. Then the `Span<byte>` can be passed to `Set`, which copies the bytes over. That way no memory is allocated on the heap.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] suchagit commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
suchagit commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r580959145



##########
File path: csharp/src/Apache.Arrow/Arrays/Decimal128Array.cs
##########
@@ -0,0 +1,96 @@
+// 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.
+
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Numerics;
+using Apache.Arrow.Arrays;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class Decimal128Array : FixedSizeBinaryArray
+    {
+        public class Builder : BuilderBase<Decimal128Array, Builder>
+        {
+            public Builder(Decimal128Type type) : base(type, 16)
+            {
+                DataType = type;
+            }
+
+            protected new Decimal128Type DataType { get; }
+
+            protected override Decimal128Array Build(ArrayData data)
+            {
+                return new Decimal128Array(data);
+            }
+
+            public Builder Append(decimal value)
+            {
+                DecimalUtility.CheckPrecisionAndScale(value, DataType.Precision, DataType.Scale, out BigInteger integerValue);
+                byte[] bytes = DecimalUtility.GetBytes(integerValue, DataType.ByteWidth);
+
+                return Append(bytes);
+            }
+
+            public Builder AppendRange(IEnumerable<decimal> values)
+            {
+                if (values == null)
+                {
+                    throw new ArgumentNullException(nameof(values));
+                }
+
+                foreach (decimal d in values)
+                {
+                    Append(d);
+                }
+
+                return Instance;
+            }
+
+            public Builder Set(int index, decimal value)
+            {
+                DecimalUtility.CheckPrecisionAndScale(value, DataType.Precision, DataType.Scale, out BigInteger integerValue);
+                byte[] bytes = DecimalUtility.GetBytes(integerValue, DataType.ByteWidth);

Review comment:
       I've had a go at this for Set and Append




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#issuecomment-807432162


   This fell of my radar, sorry! Merging.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r568754230



##########
File path: csharp/src/Apache.Arrow/Arrays/DecimalArray.cs
##########
@@ -0,0 +1,45 @@
+// 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.
+
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DecimalArray : PrimitiveArray<decimal>

Review comment:
       I don't think the C# in-memory format of decimal is the same layout as the Arrow decimal format, is it? I'm trying to find the documentation on the memory layout, the only thing I can find is in the Schema.fbs:
   
   https://github.com/apache/arrow/blob/e676aeacb2c6597c063faeed20d6142b0df938d8/format/Schema.fbs#L176-L185
   
   The C# decimal layout is different:
   
   https://docs.microsoft.com/en-us/dotnet/api/system.decimal?view=net-5.0#remarks
   
   > The binary representation of a Decimal value consists of a 1-bit sign, a 96-bit integer number, and a scaling factor used to divide the 96-bit integer and specify what portion of it is a decimal fraction. The scaling factor is implicitly the number 10, raised to an exponent ranging from 0 to 28.

##########
File path: csharp/src/Apache.Arrow/Types/DecimalType.cs
##########
@@ -17,6 +17,7 @@ namespace Apache.Arrow.Types
 {
     public sealed class DecimalType: FixedSizeBinaryType
     {
+        public static readonly DecimalType Default = new DecimalType(0, 0);

Review comment:
       I'm not sure a decimal with precision = 0 is valid.
   
   Looking at the C++ implementation, I don't see a "default" DecimalType being defined. Maybe it is best to leave it up to the caller?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r568969185



##########
File path: csharp/src/Apache.Arrow/Arrays/DecimalArray.cs
##########
@@ -0,0 +1,45 @@
+// 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.
+
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DecimalArray : PrimitiveArray<decimal>

Review comment:
       Thanks @emkornfield. One thing that I don't quite understand is the `precision` field in metadata. The data is fixed sized right? So each value will always take up 128 or 256 bits. But if I set the precision lower than `38` for a Decimal128, what exactly does that mean? Does it mean to ignore digits at the beginning of the integer? For example:
   
   ```
   Precision = 3
   Scale = 0
   Data Value = 0b010111011100   (1,500 in binary)
   ```
   
   I assume this number means `500`, and the leading `1,000` gets truncated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#issuecomment-788083728


   Can you also update 
   
   https://github.com/apache/arrow/blob/master/docs/source/status.rst
   
   To check `Decimal128` and `Decimal256` support for C#?
   
   Also, should be able to remove `Decimal` from the `Not Implemented` section of https://github.com/apache/arrow/blob/master/csharp/README.md


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r589604019



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,144 @@
+// 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.
+
+using System;
+using System.Numerics;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(decimal value, int precision, int scale, int byteWidth, Span<byte> bytes)
+        {
+            // create BigInteger from decimal
+            byte[] bigIntBytes = new byte[12];
+            int[] decimalBits = decimal.GetBits(value);
+            int decScale = (decimalBits[3] >> 16) & 0x7F;
+            for (int i = 0; i < 3; i++)
+            {
+                int bit = decimalBits[i];
+                byte[] intBytes = BitConverter.GetBytes(bit);
+                for (int j = 0; j < intBytes.Length; j++)
+                {
+                    bigIntBytes[4*i+j] =intBytes[j];
+                }
+            }
+
+            BigInteger bigInt = new BigInteger(bigIntBytes);
+            if (value < 0)
+            {
+                bigInt = -bigInt;
+            }
+
+            // validate precision and scale
+            if (decScale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + decScale + " != " + scale);

Review comment:
       ```suggestion
                   throw new OverflowException($"Decimal scale cannot be greater than that in the Arrow vector: {decScale} != {scale}");
   ```

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,144 @@
+// 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.
+
+using System;
+using System.Numerics;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)

Review comment:
       It looks like `isUnsigned` can be removed. No caller passes it in, and it doesn't appear to be used in this method.

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,144 @@
+// 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.
+
+using System;
+using System.Numerics;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(decimal value, int precision, int scale, int byteWidth, Span<byte> bytes)
+        {
+            // create BigInteger from decimal
+            byte[] bigIntBytes = new byte[12];

Review comment:
       We should be able to remove this allocation on NETCOREAPP and instead use `stackalloc byte[12]`. 

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,144 @@
+// 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.
+
+using System;
+using System.Numerics;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility

Review comment:
       Let's change this back to `internal`. We shouldn't be exposing APIs that don't need to be public. The reason is because if we make it public, we can never change it without potentially breaking someone. But if we don't make it public in the first place, consumers can't use it. So we can make any change we want in the future without worrying about breaking someone.
   
   I'm happy to see the unit tests, but this doesn't need to be public in order to have those tests. We can either:
   1. Test it through existing public APIs (like a DecimalArray)
   2. Use Reflection to call the internal methods
   3. (not recommended unless absolutely necessary) use InternalsVisibleTo

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,144 @@
+// 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.
+
+using System;
+using System.Numerics;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(decimal value, int precision, int scale, int byteWidth, Span<byte> bytes)
+        {
+            // create BigInteger from decimal
+            byte[] bigIntBytes = new byte[12];
+            int[] decimalBits = decimal.GetBits(value);
+            int decScale = (decimalBits[3] >> 16) & 0x7F;
+            for (int i = 0; i < 3; i++)
+            {
+                int bit = decimalBits[i];
+                byte[] intBytes = BitConverter.GetBytes(bit);
+                for (int j = 0; j < intBytes.Length; j++)
+                {
+                    bigIntBytes[4*i+j] =intBytes[j];
+                }
+            }
+
+            BigInteger bigInt = new BigInteger(bigIntBytes);
+            if (value < 0)
+            {
+                bigInt = -bigInt;
+            }
+
+            // validate precision and scale
+            if (decScale > scale)
+                throw new OverflowException("Decimal scale can not be greater than that in the Arrow vector: " + decScale + " != " + scale);
+
+            if (bigInt >= BigInteger.Pow(10, precision))
+                throw new OverflowException("Decimal precision can not be greater than that in the Arrow vector: " + value + " has precision > " + precision);

Review comment:
       ```suggestion
                   throw new OverflowException($"Decimal precision cannot be greater than that in the Arrow vector: {value} has precision > {precision}");
   ```

##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,144 @@
+// 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.
+
+using System;
+using System.Numerics;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility
+    {
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static void GetBytes(decimal value, int precision, int scale, int byteWidth, Span<byte> bytes)
+        {
+            // create BigInteger from decimal
+            byte[] bigIntBytes = new byte[12];
+            int[] decimalBits = decimal.GetBits(value);
+            int decScale = (decimalBits[3] >> 16) & 0x7F;
+            for (int i = 0; i < 3; i++)
+            {
+                int bit = decimalBits[i];
+                byte[] intBytes = BitConverter.GetBytes(bit);

Review comment:
       Should be able to call `BitConverter.TryWriteBytes(Span<byte>, int)` on NETCOREAPP to get rid of this allocation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r568974911



##########
File path: csharp/src/Apache.Arrow/Arrays/DecimalArray.cs
##########
@@ -0,0 +1,45 @@
+// 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.
+
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DecimalArray : PrimitiveArray<decimal>

Review comment:
       Yes, we ignore most significant bits of the integer (that would correspond to higher necessary precision).  I'm sure some code might break if those bits are populated.  But for instance we use this fact when writing to parquet: https://github.com/apache/arrow/blob/0e8de085f4afd3c9aa9c9ed05405a6c6ef8c360a/cpp/src/parquet/column_writer.cc




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r591741849



##########
File path: csharp/test/Apache.Arrow.Tests/DecimalUtilityTests.cs
##########
@@ -0,0 +1,49 @@
+// 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.
+
+using System;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class DecimalUtilityTests
+    {
+        public class Overflow
+        {
+            [Theory]
+            [InlineData(100.123, 10, 4, false)]
+            [InlineData(100.123, 6, 4, false)]
+            [InlineData(100.123, 3, 3, true)]
+            [InlineData(100.123, 10, 2, true)]
+            [InlineData(100.123, 5, 2, true)]
+            [InlineData(100.123, 5, 3, true)]
+            [InlineData(100.123, 6, 3, false)]
+            public void HasExpectedResultOrThrows(decimal d, int precision , int scale, bool shouldThrow)
+            {
+                var builder = new Decimal128Array.Builder(new Decimal128Type(precision, scale));
+
+                if (shouldThrow)
+                {
+                   Assert.Throws<OverflowException>(() => builder.Append(d));
+                }
+                else
+                {
+                    builder.Append(d);

Review comment:
       Would it be helpful to get the value back out and make sure it is the same?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#issuecomment-771344773


   I'm not familiar with the C# implementation but note decimals can now be 128 or 256 bit.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] eerhardt commented on a change in pull request #9356: ARROW-11422: [C#] add decimal support

Posted by GitBox <gi...@apache.org>.
eerhardt commented on a change in pull request #9356:
URL: https://github.com/apache/arrow/pull/9356#discussion_r580436711



##########
File path: csharp/src/Apache.Arrow/DecimalUtility.cs
##########
@@ -0,0 +1,188 @@
+// 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.
+
+using System;
+using System.Linq;
+using System.Numerics;
+using System.Runtime.InteropServices;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// This is semi-optimised best attempt at converting to / from decimal and the buffers
+    /// </summary>
+    public static class DecimalUtility
+    {
+        private static readonly byte[] _minusOne = { 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255 };
+        private static readonly BigInteger _maxDecimal = new BigInteger(decimal.MaxValue);
+        private static readonly BigInteger _minDecimal = new BigInteger(decimal.MinValue);
+        private static readonly ulong[] s_powersOfTen =
+        {
+            1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000, 10000000000, 100000000000,
+            1000000000000, 10000000000000, 100000000000000, 1000000000000000, 10000000000000000, 100000000000000000,
+            1000000000000000000, 10000000000000000000
+        };
+        private static int PowersOfTenLength => s_powersOfTen.Length - 1;
+
+
+        public static decimal GetDecimal(in ArrowBuffer valueBuffer, int index, int scale, int byteWidth,
+            bool isUnsigned = false)
+        {
+            int startIndex = index * byteWidth;
+            ReadOnlySpan<byte> value = valueBuffer.Span.Slice(startIndex, byteWidth);
+            BigInteger integerValue;
+
+#if NETCOREAPP
+            integerValue = new BigInteger(value);
+#else
+            integerValue = new BigInteger(value.ToArray());
+#endif
+
+            if (integerValue > _maxDecimal || integerValue < _minDecimal)
+            {
+                BigInteger scaleBy = BigInteger.Pow(10, scale);
+                BigInteger integerPart = BigInteger.DivRem(integerValue, scaleBy, out BigInteger fractionalPart);
+                if (integerPart > _maxDecimal || integerPart < _minDecimal) // decimal overflow, not much we can do here - C# needs a BigDecimal
+                {
+                    throw new OverflowException("Value: " + integerPart + " too big or too small to be represented as a decimal");
+                }
+                return (decimal)integerPart + DivideByScale(fractionalPart, scale);
+            }
+            else
+            {
+                return DivideByScale(integerValue, scale);
+            }
+        }
+
+        private static decimal DivideByScale(BigInteger integerValue, int scale)
+        {
+            decimal result = (decimal)integerValue; // this cast is safe here
+            int drop = scale;
+            while (drop > PowersOfTenLength)
+            {
+                result /= s_powersOfTen[PowersOfTenLength];
+                drop -= PowersOfTenLength;
+            }
+
+            result /= s_powersOfTen[drop];
+            return result;
+        }
+
+        public static byte[] GetBytes(BigInteger integerValue, int byteWidth)
+        {
+            byte[] integerBytes = integerValue.ToByteArray();
+            if (integerBytes.Length > byteWidth)
+            {
+                throw new OverflowException("Decimal size greater than " + byteWidth + " bytes: " + integerBytes.Length);
+            }
+
+            if (integerBytes.Length == byteWidth)
+                return integerBytes;
+
+            byte[] result = new byte[byteWidth];
+            if (integerValue.Sign == -1)
+            {
+                Buffer.BlockCopy(integerBytes, 0, result, 0, integerBytes.Length);
+                Buffer.BlockCopy(_minusOne, 0, result, integerBytes.Length, byteWidth - integerBytes.Length);
+            } else

Review comment:
       (nit) the `else` should be on a new line.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org