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 2020/07/08 00:21:17 UTC

[GitHub] [arrow] mr-smidge opened a new pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

mr-smidge opened a new pull request #7671:
URL: https://github.com/apache/arrow/pull/7671


   This PR fixes a few bugs in `BinaryArray.Builder()`:
   * Fixes the `Clear()` method, which previously would break all subsequently-appended values (see JIRA ticket for examples).
   * Makes the `Build()` method idempotent, by making sure the builder's internal arrays are not modified when it is called.
   
   And makes a few other improvements:
   * Comprehensive test coverage of the builder's key methods (this was written in proper TDD-style, so these tests would fail if run on `master`).
   * XML docstrings for all methods affected.


----------------------------------------------------------------
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] mr-smidge commented on a change in pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on a change in pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#discussion_r451214556



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);

Review comment:
       For an array with N items, there need to be N+1 offset values written.
   
   Previously, an offset was written with each `Append*()` call, with the extra one written when calling `Build()`.  This PR flips this around to add the extra offset upon construction (or calling `Clear()`), with the others written post-hoc during `Append*()` calls.  As such, `Build()` is now idempotent.




----------------------------------------------------------------
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 #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

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



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);

Review comment:
       Nice fix.




----------------------------------------------------------------
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] mr-smidge commented on a change in pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on a change in pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#discussion_r451216615



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -237,7 +329,9 @@ public ReadOnlySpan<byte> GetBytes(int index)
 
             if (IsNull(index))
             {
-                return null;
+                // Note that `return null;` is valid syntax, but would be misleading as `null` in the context of a span
+                // is actually returned as an empty span.
+                return ReadOnlySpan<byte>.Empty;

Review comment:
       This clause previously said `return null;`, which was misleading as it suggested that the method could be used to identify null values in the array.  However, `null` in the context of a `ReadOnlySpan<T>` is the same as the empty span, and so the method can't distinguish null values from empty ones:
   
   ```csharp
   Assert.True(ReadOnlySpan<byte>.Empty == null)
   ```
   
   The new code is clearer in intent.




----------------------------------------------------------------
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 #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

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


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


----------------------------------------------------------------
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 #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

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



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -237,7 +329,9 @@ public ReadOnlySpan<byte> GetBytes(int index)
 
             if (IsNull(index))
             {
-                return null;
+                // Note that `return null;` is valid syntax, but would be misleading as `null` in the context of a span
+                // is actually returned as an empty span.
+                return ReadOnlySpan<byte>.Empty;

Review comment:
       I've been confused and bit by this a few times in different projects. All the following ways produce an "equivalent" span:
   
   * `return null`  (this uses the implicit operator from an array)
   * `return default`
   * `return ReadOnlySpan<T>.Empty`  (this just says `return default` underneath the covers)
   
   I agree that `ReadOnlySpan<T>.Empty` is the most clear thing to return here. Next would be `return default`. `return null` is the worst of the three options IMO.

##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,

Review comment:
       Can this line just be `Length` ?




----------------------------------------------------------------
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] mr-smidge commented on a change in pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on a change in pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#discussion_r453690625



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,
+                    NullCount,
+                    offset: 0,
+                    bufs);
 
                 return Build(data);
             }
 
+            /// <summary>
+            /// Append a single null value to the array.
+            /// </summary>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendNull()
             {
-                ValueOffsets.Append(Offset);
+                // Do not add to the value buffer in the case of a null.
+                // Note that we do not need to increment the offset as a result.
                 ValidityBuffer.Append(false);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Appends a value, consisting of a single byte, to the array.
+            /// </summary>
+            /// <param name="value">Byte value to append.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(byte value)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(value);
-                Offset++;
                 ValidityBuffer.Append(true);
+                Offset++;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append a value, consisting of a span of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that a single value is added, which consists of arbitrarily many bytes.  If multiple values are
+            /// to be added, use the <see cref="AppendRange"/> method.
+            /// </remarks>
+            /// <param name="span">Span of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(ReadOnlySpan<byte> span)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(span);
                 ValidityBuffer.Append(true);
                 Offset += span.Length;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
-            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            /// <summary>
+            /// Append a value, consisting of an enumerable collection of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends a single value, which may consist of arbitrarily many bytes.  If multiple
+            /// values are to be added, use the <see cref="AppendRange(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="value">Enumerable collection of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder Append(IEnumerable<byte> value)
             {
-                foreach (byte[] arr in values)
+                if (value == null)
                 {
-                    if (arr == null)
-                    {
-                        AppendNull();
-                        continue;
-                    }
-                    int len = ValueBuffer.Length;
-                    ValueOffsets.Append(Offset);
-                    ValueBuffer.Append(arr);
-                    ValidityBuffer.Append(true);
-                    Offset += ValueBuffer.Length - len;
+                    return AppendNull();
                 }
 
+                // Note: by looking at the length of the value buffer before and after, we avoid having to iterate
+                // through the enumerable multiple times to get both length and contents.
+                int priorLength = ValueBuffer.Length;
+                ValueBuffer.AppendRange(value);
+                int valueLength = ValueBuffer.Length - priorLength;
+                Offset += valueLength;
+                ValidityBuffer.Append(true);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append an enumerable collection of single-byte values to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends multiple values, each of which is a single byte.  If a single value is
+            /// to be added, use the <see cref="Append(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="values">Single-byte values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendRange(IEnumerable<byte> values)
             {
-                if (values == null)
+                foreach (byte b in values)

Review comment:
       Fixed.




----------------------------------------------------------------
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 #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

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



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,
+                    NullCount,
+                    offset: 0,
+                    bufs);
 
                 return Build(data);
             }
 
+            /// <summary>
+            /// Append a single null value to the array.
+            /// </summary>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendNull()
             {
-                ValueOffsets.Append(Offset);
+                // Do not add to the value buffer in the case of a null.
+                // Note that we do not need to increment the offset as a result.
                 ValidityBuffer.Append(false);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Appends a value, consisting of a single byte, to the array.
+            /// </summary>
+            /// <param name="value">Byte value to append.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(byte value)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(value);
-                Offset++;
                 ValidityBuffer.Append(true);
+                Offset++;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append a value, consisting of a span of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that a single value is added, which consists of arbitrarily many bytes.  If multiple values are
+            /// to be added, use the <see cref="AppendRange"/> method.
+            /// </remarks>
+            /// <param name="span">Span of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(ReadOnlySpan<byte> span)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(span);
                 ValidityBuffer.Append(true);
                 Offset += span.Length;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
-            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            /// <summary>
+            /// Append a value, consisting of an enumerable collection of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends a single value, which may consist of arbitrarily many bytes.  If multiple
+            /// values are to be added, use the <see cref="AppendRange(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="value">Enumerable collection of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder Append(IEnumerable<byte> value)
             {
-                foreach (byte[] arr in values)
+                if (value == null)
                 {
-                    if (arr == null)
-                    {
-                        AppendNull();
-                        continue;
-                    }
-                    int len = ValueBuffer.Length;
-                    ValueOffsets.Append(Offset);
-                    ValueBuffer.Append(arr);
-                    ValidityBuffer.Append(true);
-                    Offset += ValueBuffer.Length - len;
+                    return AppendNull();
                 }
 
+                // Note: by looking at the length of the value buffer before and after, we avoid having to iterate
+                // through the enumerable multiple times to get both length and contents.
+                int priorLength = ValueBuffer.Length;
+                ValueBuffer.AppendRange(value);
+                int valueLength = ValueBuffer.Length - priorLength;
+                Offset += valueLength;
+                ValidityBuffer.Append(true);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append an enumerable collection of single-byte values to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends multiple values, each of which is a single byte.  If a single value is
+            /// to be added, use the <see cref="Append(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="values">Single-byte values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendRange(IEnumerable<byte> values)
             {
-                if (values == null)
+                foreach (byte b in values)
                 {
-                    return AppendNull();
+                    Append(b);
                 }
-                int len = ValueBuffer.Length;
-                ValueBuffer.AppendRange(values);
-                int valOffset = ValueBuffer.Length - len;
-                ValueOffsets.Append(Offset);
-                Offset += valOffset;
-                ValidityBuffer.Append(true);
+
+                return Instance;
+            }
+
+            /// <summary>
+            /// Append an enumerable collection of values to the array.
+            /// </summary>
+            /// <param name="values">Values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            {
+                foreach (byte[] arr in values)

Review comment:
       This will throw a `NullReferenceException` if `values` is `null`.

##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,
+                    NullCount,
+                    offset: 0,
+                    bufs);
 
                 return Build(data);
             }
 
+            /// <summary>
+            /// Append a single null value to the array.
+            /// </summary>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendNull()
             {
-                ValueOffsets.Append(Offset);
+                // Do not add to the value buffer in the case of a null.
+                // Note that we do not need to increment the offset as a result.
                 ValidityBuffer.Append(false);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Appends a value, consisting of a single byte, to the array.
+            /// </summary>
+            /// <param name="value">Byte value to append.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(byte value)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(value);
-                Offset++;
                 ValidityBuffer.Append(true);
+                Offset++;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append a value, consisting of a span of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that a single value is added, which consists of arbitrarily many bytes.  If multiple values are
+            /// to be added, use the <see cref="AppendRange"/> method.
+            /// </remarks>
+            /// <param name="span">Span of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(ReadOnlySpan<byte> span)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(span);
                 ValidityBuffer.Append(true);
                 Offset += span.Length;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
-            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            /// <summary>
+            /// Append a value, consisting of an enumerable collection of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends a single value, which may consist of arbitrarily many bytes.  If multiple
+            /// values are to be added, use the <see cref="AppendRange(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="value">Enumerable collection of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder Append(IEnumerable<byte> value)
             {
-                foreach (byte[] arr in values)
+                if (value == null)
                 {
-                    if (arr == null)
-                    {
-                        AppendNull();
-                        continue;
-                    }
-                    int len = ValueBuffer.Length;
-                    ValueOffsets.Append(Offset);
-                    ValueBuffer.Append(arr);
-                    ValidityBuffer.Append(true);
-                    Offset += ValueBuffer.Length - len;
+                    return AppendNull();
                 }
 
+                // Note: by looking at the length of the value buffer before and after, we avoid having to iterate
+                // through the enumerable multiple times to get both length and contents.
+                int priorLength = ValueBuffer.Length;
+                ValueBuffer.AppendRange(value);
+                int valueLength = ValueBuffer.Length - priorLength;
+                Offset += valueLength;
+                ValidityBuffer.Append(true);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append an enumerable collection of single-byte values to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends multiple values, each of which is a single byte.  If a single value is
+            /// to be added, use the <see cref="Append(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="values">Single-byte values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendRange(IEnumerable<byte> values)
             {
-                if (values == null)
+                foreach (byte b in values)

Review comment:
       This will throw a `NullReferenceException` if `values` is `null`.

##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,
+                    NullCount,
+                    offset: 0,
+                    bufs);
 
                 return Build(data);
             }
 
+            /// <summary>
+            /// Append a single null value to the array.
+            /// </summary>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendNull()
             {
-                ValueOffsets.Append(Offset);
+                // Do not add to the value buffer in the case of a null.
+                // Note that we do not need to increment the offset as a result.
                 ValidityBuffer.Append(false);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Appends a value, consisting of a single byte, to the array.
+            /// </summary>
+            /// <param name="value">Byte value to append.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(byte value)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(value);
-                Offset++;
                 ValidityBuffer.Append(true);
+                Offset++;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append a value, consisting of a span of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that a single value is added, which consists of arbitrarily many bytes.  If multiple values are
+            /// to be added, use the <see cref="AppendRange"/> method.
+            /// </remarks>
+            /// <param name="span">Span of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(ReadOnlySpan<byte> span)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(span);
                 ValidityBuffer.Append(true);
                 Offset += span.Length;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
-            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            /// <summary>
+            /// Append a value, consisting of an enumerable collection of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends a single value, which may consist of arbitrarily many bytes.  If multiple
+            /// values are to be added, use the <see cref="AppendRange(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="value">Enumerable collection of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder Append(IEnumerable<byte> value)
             {
-                foreach (byte[] arr in values)
+                if (value == null)
                 {
-                    if (arr == null)
-                    {
-                        AppendNull();
-                        continue;
-                    }
-                    int len = ValueBuffer.Length;
-                    ValueOffsets.Append(Offset);
-                    ValueBuffer.Append(arr);
-                    ValidityBuffer.Append(true);
-                    Offset += ValueBuffer.Length - len;
+                    return AppendNull();
                 }
 
+                // Note: by looking at the length of the value buffer before and after, we avoid having to iterate
+                // through the enumerable multiple times to get both length and contents.
+                int priorLength = ValueBuffer.Length;
+                ValueBuffer.AppendRange(value);
+                int valueLength = ValueBuffer.Length - priorLength;
+                Offset += valueLength;
+                ValidityBuffer.Append(true);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append an enumerable collection of single-byte values to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends multiple values, each of which is a single byte.  If a single value is
+            /// to be added, use the <see cref="Append(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="values">Single-byte values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendRange(IEnumerable<byte> values)
             {
-                if (values == null)
+                foreach (byte b in values)
                 {
-                    return AppendNull();
+                    Append(b);
                 }
-                int len = ValueBuffer.Length;
-                ValueBuffer.AppendRange(values);
-                int valOffset = ValueBuffer.Length - len;
-                ValueOffsets.Append(Offset);
-                Offset += valOffset;
-                ValidityBuffer.Append(true);
+
+                return Instance;
+            }
+
+            /// <summary>
+            /// Append an enumerable collection of values to the array.
+            /// </summary>
+            /// <param name="values">Values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            {
+                foreach (byte[] arr in values)
+                {
+                    Append((IEnumerable<byte>)arr);

Review comment:
       This should call `Append(ReadOnlySpan<byte> span)` instead. It is faster. You can easily check for `null` and call `AppendNull()` if `arr` is null.




----------------------------------------------------------------
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] mr-smidge commented on a change in pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on a change in pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#discussion_r453692211



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,

Review comment:
       Yes it can - changed.




----------------------------------------------------------------
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 #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

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


   


----------------------------------------------------------------
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] mr-smidge commented on a change in pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on a change in pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#discussion_r453690865



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,
+                    NullCount,
+                    offset: 0,
+                    bufs);
 
                 return Build(data);
             }
 
+            /// <summary>
+            /// Append a single null value to the array.
+            /// </summary>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendNull()
             {
-                ValueOffsets.Append(Offset);
+                // Do not add to the value buffer in the case of a null.
+                // Note that we do not need to increment the offset as a result.
                 ValidityBuffer.Append(false);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Appends a value, consisting of a single byte, to the array.
+            /// </summary>
+            /// <param name="value">Byte value to append.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(byte value)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(value);
-                Offset++;
                 ValidityBuffer.Append(true);
+                Offset++;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append a value, consisting of a span of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that a single value is added, which consists of arbitrarily many bytes.  If multiple values are
+            /// to be added, use the <see cref="AppendRange"/> method.
+            /// </remarks>
+            /// <param name="span">Span of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(ReadOnlySpan<byte> span)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(span);
                 ValidityBuffer.Append(true);
                 Offset += span.Length;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
-            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            /// <summary>
+            /// Append a value, consisting of an enumerable collection of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends a single value, which may consist of arbitrarily many bytes.  If multiple
+            /// values are to be added, use the <see cref="AppendRange(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="value">Enumerable collection of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder Append(IEnumerable<byte> value)
             {
-                foreach (byte[] arr in values)
+                if (value == null)
                 {
-                    if (arr == null)
-                    {
-                        AppendNull();
-                        continue;
-                    }
-                    int len = ValueBuffer.Length;
-                    ValueOffsets.Append(Offset);
-                    ValueBuffer.Append(arr);
-                    ValidityBuffer.Append(true);
-                    Offset += ValueBuffer.Length - len;
+                    return AppendNull();
                 }
 
+                // Note: by looking at the length of the value buffer before and after, we avoid having to iterate
+                // through the enumerable multiple times to get both length and contents.
+                int priorLength = ValueBuffer.Length;
+                ValueBuffer.AppendRange(value);
+                int valueLength = ValueBuffer.Length - priorLength;
+                Offset += valueLength;
+                ValidityBuffer.Append(true);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append an enumerable collection of single-byte values to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends multiple values, each of which is a single byte.  If a single value is
+            /// to be added, use the <see cref="Append(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="values">Single-byte values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendRange(IEnumerable<byte> values)
             {
-                if (values == null)
+                foreach (byte b in values)
                 {
-                    return AppendNull();
+                    Append(b);
                 }
-                int len = ValueBuffer.Length;
-                ValueBuffer.AppendRange(values);
-                int valOffset = ValueBuffer.Length - len;
-                ValueOffsets.Append(Offset);
-                Offset += valOffset;
-                ValidityBuffer.Append(true);
+
+                return Instance;
+            }
+
+            /// <summary>
+            /// Append an enumerable collection of values to the array.
+            /// </summary>
+            /// <param name="values">Values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            {
+                foreach (byte[] arr in values)
+                {
+                    Append((IEnumerable<byte>)arr);

Review comment:
       Thanks - changed.




----------------------------------------------------------------
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] mr-smidge commented on a change in pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on a change in pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#discussion_r451215646



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -173,11 +245,19 @@ public TBuilder Set(int index, byte value)
                 throw new NotImplementedException();
             }
 
+            /// <summary>
+            /// Clear all contents appended so far.
+            /// </summary>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Clear()
             {
                 ValueOffsets.Clear();
                 ValueBuffer.Clear();
                 ValidityBuffer.Clear();
+
+                // Always write the first offset before anything has been written.
+                Offset = 0;

Review comment:
       Not resetting the `Offset` member variable was the root cause of the string-clearing bug referenced in ARROW-8344.




----------------------------------------------------------------
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] mr-smidge commented on a change in pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on a change in pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#discussion_r451215414



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,
+                    NullCount,
+                    offset: 0,
+                    bufs);
 
                 return Build(data);
             }
 
+            /// <summary>
+            /// Append a single null value to the array.
+            /// </summary>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendNull()
             {
-                ValueOffsets.Append(Offset);
+                // Do not add to the value buffer in the case of a null.
+                // Note that we do not need to increment the offset as a result.
                 ValidityBuffer.Append(false);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Appends a value, consisting of a single byte, to the array.
+            /// </summary>
+            /// <param name="value">Byte value to append.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(byte value)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(value);
-                Offset++;
                 ValidityBuffer.Append(true);
+                Offset++;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append a value, consisting of a span of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that a single value is added, which consists of arbitrarily many bytes.  If multiple values are
+            /// to be added, use the <see cref="AppendRange"/> method.
+            /// </remarks>
+            /// <param name="span">Span of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(ReadOnlySpan<byte> span)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(span);
                 ValidityBuffer.Append(true);
                 Offset += span.Length;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
-            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            /// <summary>
+            /// Append a value, consisting of an enumerable collection of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends a single value, which may consist of arbitrarily many bytes.  If multiple
+            /// values are to be added, use the <see cref="AppendRange(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="value">Enumerable collection of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder Append(IEnumerable<byte> value)
             {
-                foreach (byte[] arr in values)
+                if (value == null)
                 {
-                    if (arr == null)
-                    {
-                        AppendNull();
-                        continue;
-                    }
-                    int len = ValueBuffer.Length;
-                    ValueOffsets.Append(Offset);
-                    ValueBuffer.Append(arr);
-                    ValidityBuffer.Append(true);
-                    Offset += ValueBuffer.Length - len;
+                    return AppendNull();
                 }
 
+                // Note: by looking at the length of the value buffer before and after, we avoid having to iterate
+                // through the enumerable multiple times to get both length and contents.
+                int priorLength = ValueBuffer.Length;
+                ValueBuffer.AppendRange(value);
+                int valueLength = ValueBuffer.Length - priorLength;
+                Offset += valueLength;
+                ValidityBuffer.Append(true);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append an enumerable collection of single-byte values to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends multiple values, each of which is a single byte.  If a single value is
+            /// to be added, use the <see cref="Append(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="values">Single-byte values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendRange(IEnumerable<byte> values)

Review comment:
       This `AppendRange(IEnumerable<byte>)` method previously did the "wrong thing": it was called `AppendRange()`, but it didn't add a range of values.  Instead, it added a single value.
   
   This method now does what one would expect when calling `AppendRange()`, and adds a range of values (each of which is a single byte).  The original method behaviour is now available via `Append(IEnumerable<byte>)` instead.




----------------------------------------------------------------
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] mr-smidge commented on pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#issuecomment-657595775


   > This is looking really good. I just have a few comments, and then it should be ready to merge.
   
   All comments should be addressed now :smile:.


----------------------------------------------------------------
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] mr-smidge commented on a change in pull request #7671: ARROW-8344: [C#] Bug-fixes to binary array plus other improvements

Posted by GitBox <gi...@apache.org>.
mr-smidge commented on a change in pull request #7671:
URL: https://github.com/apache/arrow/pull/7671#discussion_r453690533



##########
File path: csharp/src/Apache.Arrow/Arrays/BinaryArray.cs
##########
@@ -66,87 +66,158 @@ protected BuilderBase(IArrowType dataType)
                 ValueOffsets = new ArrowBuffer.Builder<int>();
                 ValueBuffer = new ArrowBuffer.Builder<byte>();
                 ValidityBuffer = new ArrowBuffer.BitmapBuilder();
+
+                // From the docs:
+                //
+                // The offsets buffer contains length + 1 signed integers (either 32-bit or 64-bit, depending on the
+                // logical type), which encode the start position of each slot in the data buffer. The length of the
+                // value in each slot is computed using the difference between the offset at that slot’s index and the
+                // subsequent offset.
+                //
+                // In this builder, we choose to append the first offset (zero) upon construction, and each trailing
+                // offset is then added after each individual item has been appended.
+                ValueOffsets.Append(this.Offset);
             }
 
             protected abstract TArray Build(ArrayData data);
 
-            public int Length => ValueOffsets.Length;
+            /// <summary>
+            /// Gets the length of the array built so far.
+            /// </summary>
+            public int Length => ValueOffsets.Length - 1;
 
+            /// <summary>
+            /// Build an Arrow array from the appended contents so far.
+            /// </summary>
+            /// <param name="allocator">Optional memory allocator.</param>
+            /// <returns>Returns an array of type <typeparamref name="TArray"/>.</returns>
             public TArray Build(MemoryAllocator allocator = default)
             {
-                ValueOffsets.Append(Offset);
-
-                ArrowBuffer validityBuffer = NullCount > 0
-                                        ? ValidityBuffer.Build(allocator)
-                                        : ArrowBuffer.Empty;
-
-                var data = new ArrayData(DataType, ValueOffsets.Length - 1, NullCount, 0,
-                    new[] { validityBuffer, ValueOffsets.Build(allocator), ValueBuffer.Build(allocator) });
+                var bufs = new[]
+                {
+                    NullCount > 0 ? ValidityBuffer.Build(allocator) : ArrowBuffer.Empty,
+                    ValueOffsets.Build(allocator),
+                    ValueBuffer.Build(allocator),
+                };
+                var data = new ArrayData(
+                    DataType,
+                    length: ValueOffsets.Length - 1,
+                    NullCount,
+                    offset: 0,
+                    bufs);
 
                 return Build(data);
             }
 
+            /// <summary>
+            /// Append a single null value to the array.
+            /// </summary>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendNull()
             {
-                ValueOffsets.Append(Offset);
+                // Do not add to the value buffer in the case of a null.
+                // Note that we do not need to increment the offset as a result.
                 ValidityBuffer.Append(false);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Appends a value, consisting of a single byte, to the array.
+            /// </summary>
+            /// <param name="value">Byte value to append.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(byte value)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(value);
-                Offset++;
                 ValidityBuffer.Append(true);
+                Offset++;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append a value, consisting of a span of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that a single value is added, which consists of arbitrarily many bytes.  If multiple values are
+            /// to be added, use the <see cref="AppendRange"/> method.
+            /// </remarks>
+            /// <param name="span">Span of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder Append(ReadOnlySpan<byte> span)
             {
-                ValueOffsets.Append(Offset);
                 ValueBuffer.Append(span);
                 ValidityBuffer.Append(true);
                 Offset += span.Length;
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
-            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            /// <summary>
+            /// Append a value, consisting of an enumerable collection of bytes, to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends a single value, which may consist of arbitrarily many bytes.  If multiple
+            /// values are to be added, use the <see cref="AppendRange(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="value">Enumerable collection of bytes to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder Append(IEnumerable<byte> value)
             {
-                foreach (byte[] arr in values)
+                if (value == null)
                 {
-                    if (arr == null)
-                    {
-                        AppendNull();
-                        continue;
-                    }
-                    int len = ValueBuffer.Length;
-                    ValueOffsets.Append(Offset);
-                    ValueBuffer.Append(arr);
-                    ValidityBuffer.Append(true);
-                    Offset += ValueBuffer.Length - len;
+                    return AppendNull();
                 }
 
+                // Note: by looking at the length of the value buffer before and after, we avoid having to iterate
+                // through the enumerable multiple times to get both length and contents.
+                int priorLength = ValueBuffer.Length;
+                ValueBuffer.AppendRange(value);
+                int valueLength = ValueBuffer.Length - priorLength;
+                Offset += valueLength;
+                ValidityBuffer.Append(true);
+                ValueOffsets.Append(Offset);
                 return Instance;
             }
 
+            /// <summary>
+            /// Append an enumerable collection of single-byte values to the array.
+            /// </summary>
+            /// <remarks>
+            /// Note that this method appends multiple values, each of which is a single byte.  If a single value is
+            /// to be added, use the <see cref="Append(IEnumerable{byte})"/> method instead.
+            /// </remarks>
+            /// <param name="values">Single-byte values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
             public TBuilder AppendRange(IEnumerable<byte> values)
             {
-                if (values == null)
+                foreach (byte b in values)
                 {
-                    return AppendNull();
+                    Append(b);
                 }
-                int len = ValueBuffer.Length;
-                ValueBuffer.AppendRange(values);
-                int valOffset = ValueBuffer.Length - len;
-                ValueOffsets.Append(Offset);
-                Offset += valOffset;
-                ValidityBuffer.Append(true);
+
+                return Instance;
+            }
+
+            /// <summary>
+            /// Append an enumerable collection of values to the array.
+            /// </summary>
+            /// <param name="values">Values to add.</param>
+            /// <returns>Returns the builder (for fluent-style composition).</returns>
+            public TBuilder AppendRange(IEnumerable<byte[]> values)
+            {
+                foreach (byte[] arr in values)

Review comment:
       Fixed.




----------------------------------------------------------------
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