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/10/05 19:14:02 UTC

[GitHub] [arrow] pgovind opened a new pull request #8348: C# support for StructArrays

pgovind opened a new pull request #8348:
URL: https://github.com/apache/arrow/pull/8348


   Fix for https://issues.apache.org/jira/browse/ARROW-6972
   
   Add support for `StructArray`. It'd be nice to have an easier way to construct a `StructArray`, but that can come in a follow up PR


----------------------------------------------------------------
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] pgovind commented on a change in pull request #8348: ARROW-6972: [C#] Support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Ipc/MessageSerializer.cs
##########
@@ -57,16 +58,30 @@ internal static Schema GetSchema(Flatbuf.Schema schema)
             for (int i = 0; i < schema.FieldsLength; i++)
             {
                 Flatbuf.Field field = schema.Fields(i).GetValueOrDefault();
-
-                schemaBuilder.Field(
-                    new Field(field.Name, GetFieldArrowType(field), field.Nullable));
+                Field arrowField = FieldFromFlatbuffer(field);
+                schemaBuilder.Field(arrowField);
             }
 
             return schemaBuilder.Build();
         }
 
+        private static Field FieldFromFlatbuffer(Flatbuf.Field flatbufField)
+        {
+            System.Collections.Generic.List<Field> childFields = null;
+            if (flatbufField.ChildrenLength > 0)
+            {
+                childFields = new System.Collections.Generic.List<Field>();

Review comment:
       Oh yea you're right. IReadOnlyList only needs `this[index]`. Fixed 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] pgovind commented on a change in pull request #8348: ARROW-6972: [C#] C# support for StructArrays

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrayBuilderTests.cs
##########
@@ -73,6 +73,67 @@ public void StringArrayBuilderHandlesNullsAndEmptyStrings()
             Assert.Equal(string.Empty, stringArray.GetString(3));
         }
 
+        [Fact]
+        public void TestStructArray()
+        {
+            // The following can be improved with a Builder class for StructArray.
+            List<Field> fields = new List<Field>();
+            Field.Builder fieldBuilder = new Field.Builder();
+            fields.Add(fieldBuilder.Name("Strings").DataType(StringType.Default).Nullable(true).Build());
+            fieldBuilder = new Field.Builder();
+            fields.Add(fieldBuilder.Name("Ints").DataType(Int32Type.Default).Nullable(true).Build());
+            StructType structType = new StructType(fields);
+
+            StringArray.Builder stringBuilder = new StringArray.Builder();
+            StringArray stringArray = stringBuilder.Append("joe").AppendNull().AppendNull().Append("mark").Build();
+            Int32Array.Builder intBuilder = new Int32Array.Builder();
+            Int32Array intArray = intBuilder.Append(1).Append(2).AppendNull().Append(4).Build();
+            List<Array> arrays = new List<Array>();
+            arrays.Add(stringArray);
+            arrays.Add(intArray);
+
+            ArrowBuffer.BitmapBuilder nullBitmap = new ArrowBuffer.BitmapBuilder();
+            var nullBitmapBuffer = nullBitmap.Append(true).Append(true).Append(false).Append(true).Build();
+            StructArray structs = new StructArray(structType, 4, arrays, nullBitmapBuffer, 1);
+
+            Assert.Equal(4, structs.Length);

Review comment:
       Replace the following asserts with ArrayComparer




----------------------------------------------------------------
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] nealrichardson commented on pull request #8348: ARROW-6972: [C#] Support for StructArrays

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


   @eerhardt I haven't looked but I think the authors of that new workflow are working to fix it. You can ignore it.


----------------------------------------------------------------
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 #8348: ARROW-6972: [C#] C# support for StructArrays

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrayBuilderTests.cs
##########
@@ -73,7 +73,6 @@ public void StringArrayBuilderHandlesNullsAndEmptyStrings()
             Assert.Equal(string.Empty, stringArray.GetString(3));
         }
 
-

Review comment:
       Can you revert this change, since it is the only one in this file?

##########
File path: csharp/src/Apache.Arrow/Arrays/StructArray.cs
##########
@@ -21,13 +21,13 @@ namespace Apache.Arrow
 {
     public class StructArray : Array
     {
-        private readonly List<Array> _fields;
+        private readonly IEnumerable<IArrowArray> _fields;

Review comment:
       A couple of questions here:
   
   1. Why the move from `Array` to `IArrowArray`? I kind of think we should remove `IArrowArray`, since the interface doesn't provide any benefit over the abstract class. See https://issues.apache.org/jira/browse/ARROW-5546
   2. How does this `_fields` property ever contain anything? AFAICT - it will always be an empty List.

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -180,9 +193,15 @@ internal static ByteBuffer CreateByteBuffer(ReadOnlyMemory<byte> buffer)
         {
 
             ArrowBuffer nullArrowBuffer = BuildArrowBuffer(bodyData, recordBatchEnumerator.CurrentBuffer);
-            recordBatchEnumerator.MoveNextBuffer();
+            if (!recordBatchEnumerator.MoveNextBuffer())

Review comment:
       👍 

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -165,8 +165,21 @@ internal static ByteBuffer CreateByteBuffer(ReadOnlyMemory<byte> buffer)
                 throw new InvalidDataException("Null count length must be >= 0"); // TODO:Localize exception message
             }
 
-            ArrowBuffer[] arrowBuff = new[] { nullArrowBuffer, valueArrowBuffer };
-            ArrayData[] children = GetChildren(ref recordBatchEnumerator, field, bodyData);
+            ArrowBuffer[] arrowBuff = null;
+            ArrayData[] children = null;
+            if (field.DataType.TypeId == ArrowTypeId.Struct)
+            {
+                arrowBuff = new[] { nullArrowBuffer};
+                children = GetChildren(ref recordBatchEnumerator, field, bodyData);
+            }
+            else
+            {
+                ArrowBuffer valueArrowBuffer = BuildArrowBuffer(bodyData, recordBatchEnumerator.CurrentBuffer);
+                recordBatchEnumerator.MoveNextBuffer();
+
+                arrowBuff = new[] { nullArrowBuffer, valueArrowBuffer };
+                children = GetChildren(ref recordBatchEnumerator, field, bodyData);

Review comment:
       it looks like the `children` line can be refactored out of the `if`. It looks the same for both cases.

##########
File path: csharp/src/Apache.Arrow/Ipc/MessageSerializer.cs
##########
@@ -57,16 +58,30 @@ internal static Schema GetSchema(Flatbuf.Schema schema)
             for (int i = 0; i < schema.FieldsLength; i++)
             {
                 Flatbuf.Field field = schema.Fields(i).GetValueOrDefault();
-
-                schemaBuilder.Field(
-                    new Field(field.Name, GetFieldArrowType(field), field.Nullable));
+                Field arrowField = FieldFromFlatbuffer(field);
+                schemaBuilder.Field(arrowField);
             }
 
             return schemaBuilder.Build();
         }
 
+        private static Field FieldFromFlatbuffer(Flatbuf.Field flatbufField)
+        {
+            System.Collections.Generic.List<Field> childFields = null;
+            if (flatbufField.ChildrenLength > 0)
+            {
+                childFields = new System.Collections.Generic.List<Field>();

Review comment:
       ```suggestion
                   childFields = new System.Collections.Generic.List<Field>(flatbufField.ChildrenLength);
   ```
   
   Since we know up front how many we will need.
   
   Actually, do we even need a `List` here? Could it simply be an array since we know how many we will need?

##########
File path: csharp/src/Apache.Arrow/Types/StructType.cs
##########
@@ -28,7 +28,7 @@ public sealed class StructType : ArrowType
 
         public IEnumerable<Field> Fields => _fields;

Review comment:
       Since we are deriving from `NestedType` now, why do we need the `Fields` property? `NestedType` already has a `Children` property, which will be the same thing.

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -165,8 +165,21 @@ internal static ByteBuffer CreateByteBuffer(ReadOnlyMemory<byte> buffer)
                 throw new InvalidDataException("Null count length must be >= 0"); // TODO:Localize exception message
             }
 
-            ArrowBuffer[] arrowBuff = new[] { nullArrowBuffer, valueArrowBuffer };
-            ArrayData[] children = GetChildren(ref recordBatchEnumerator, field, bodyData);
+            ArrowBuffer[] arrowBuff = null;

Review comment:
       No need to initialize these 2 variables to `null` since they will always be set in the `if-else`.
   
   1. It is a minor perf improvement.
   2. It allows the compiler to warn you in case you didn't set them.

##########
File path: csharp/test/Apache.Arrow.Tests/StructArrayTests.cs
##########
@@ -0,0 +1,86 @@
+// 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;
+using System.Collections.Generic;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class StructArrayTests
+    {
+        [Fact]
+        public void TestStructArray()
+        {
+            // The following can be improved with a Builder class for StructArray.
+            List<Field> fields = new List<Field>();
+            Field.Builder fieldBuilder = new Field.Builder();
+            fields.Add(fieldBuilder.Name("Strings").DataType(StringType.Default).Nullable(true).Build());
+            fieldBuilder = new Field.Builder();
+            fields.Add(fieldBuilder.Name("Ints").DataType(Int32Type.Default).Nullable(true).Build());
+            StructType structType = new StructType(fields);
+
+            StringArray.Builder stringBuilder = new StringArray.Builder();
+            StringArray stringArray = stringBuilder.Append("joe").AppendNull().AppendNull().Append("mark").Build();
+            Int32Array.Builder intBuilder = new Int32Array.Builder();
+            Int32Array intArray = intBuilder.Append(1).Append(2).AppendNull().Append(4).Build();
+            List<Array> arrays = new List<Array>();
+            arrays.Add(stringArray);
+            arrays.Add(intArray);
+
+            ArrowBuffer.BitmapBuilder nullBitmap = new ArrowBuffer.BitmapBuilder();
+            var nullBitmapBuffer = nullBitmap.Append(true).Append(true).Append(false).Append(true).Build();
+            StructArray structs = new StructArray(structType, 4, arrays, nullBitmapBuffer, 1);
+
+            Assert.Equal(4, structs.Length);
+            Assert.Equal(1, structs.NullCount);
+            ArrayData[] childArrays = structs.Data.Children; // Data for StringArray and Int32Array
+            Assert.Equal(2, childArrays.Length);
+            for (int j = 0; j < childArrays.Length; j++)

Review comment:
       (nit) Why `j`? Why not `i`?

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowTypeFlatbufferBuilder.cs
##########
@@ -171,6 +172,13 @@ public void Visit(Time64Type type)
                     Flatbuf.Time.CreateTime(Builder, ToFlatBuffer(type.Unit), 64));
             }
 
+            public void Visit(StructType type)
+            {
+                Flatbuf.Struct_.StartStruct_(Builder);
+                FieldType result = FieldType.Build(Flatbuf.Type.Struct_, Flatbuf.Struct_.EndStruct_(Builder));

Review comment:
       (super nit) Do you need the `FieldType result` variable? Why not just assign `Result` directly, like the other methods do?

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowStreamWriter.cs
##########
@@ -102,6 +103,31 @@ public void Visit(BinaryArray array)
                 _buffers.Add(CreateBuffer(array.ValueBuffer));
             }
 
+            private void Visit(ArrayData[] children)
+            {
+                for (int i = 0; i < children.Length; i++)
+                {
+                    Visit(ArrowArrayFactory.BuildArray(children[i]));

Review comment:
       It's a bit unfortunate that we need to allocate a new `Array` object here. But we can get this merged as functional, and tweak the performance later.

##########
File path: csharp/src/Apache.Arrow/Types/StructType.cs
##########
@@ -28,7 +28,7 @@ public sealed class StructType : ArrowType
 
         public IEnumerable<Field> Fields => _fields;

Review comment:
       Note that C++ standardized on the "Field" name: https://github.com/apache/arrow/pull/7132




----------------------------------------------------------------
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 #8348: ARROW-6972: [C#] C# support for StructArrays

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


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


----------------------------------------------------------------
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] pgovind commented on a change in pull request #8348: ARROW-6972: [C#] C# support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Arrays/StructArray.cs
##########
@@ -21,13 +21,13 @@ namespace Apache.Arrow
 {
     public class StructArray : Array
     {
-        private readonly List<Array> _fields;
+        private readonly IEnumerable<IArrowArray> _fields;

Review comment:
       1. I agree. I had `Array` first, but I was forced to make it `IArrowArray` because `RecordBatch.Column(i)` returns an `IArrowArray`. See https://github.com/dotnet/spark/blob/7bcd2a5060e12ca993132f4986b2dfb4b07df94f/src/csharp/Microsoft.Spark.E2ETest/IpcTests/Sql/DataFrameTests.cs#L365. When we refactor `IArrowArray` -> `ArrowArray`, this can change too.
   2. Yeap, you're right. Removed it. It is unused at the moment.




----------------------------------------------------------------
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] pgovind commented on a change in pull request #8348: ARROW-6972: [C#] C# support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Ipc/MessageSerializer.cs
##########
@@ -57,16 +58,30 @@ internal static Schema GetSchema(Flatbuf.Schema schema)
             for (int i = 0; i < schema.FieldsLength; i++)
             {
                 Flatbuf.Field field = schema.Fields(i).GetValueOrDefault();
-
-                schemaBuilder.Field(
-                    new Field(field.Name, GetFieldArrowType(field), field.Nullable));
+                Field arrowField = FieldFromFlatbuffer(field);
+                schemaBuilder.Field(arrowField);
             }
 
             return schemaBuilder.Build();
         }
 
+        private static Field FieldFromFlatbuffer(Flatbuf.Field flatbufField)
+        {
+            System.Collections.Generic.List<Field> childFields = null;
+            if (flatbufField.ChildrenLength > 0)
+            {
+                childFields = new System.Collections.Generic.List<Field>();

Review comment:
       Passing in an Array here would get converted to a `List` here anyway in the `StructType` constructor. I did make the change to specify the capacity upfront 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] github-actions[bot] commented on pull request #8348: C# support for StructArrays

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


   <!--
     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] pgovind commented on a change in pull request #8348: C# support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Arrays/ArrowArrayBuilderFactory.cs
##########
@@ -59,6 +59,7 @@ static class ArrowArrayBuilderFactory
                 case ArrowTypeId.List:
                     return new ListArray.Builder(dataType as ListType);
                 case ArrowTypeId.Struct:
+                    return new StructArray.Builder(dataType as StructType);

Review comment:
       Just a bad commit. Will push the right one shortly.




----------------------------------------------------------------
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] pgovind commented on a change in pull request #8348: ARROW-6972: [C#] C# support for StructArrays

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrayBuilderTests.cs
##########
@@ -73,6 +73,67 @@ public void StringArrayBuilderHandlesNullsAndEmptyStrings()
             Assert.Equal(string.Empty, stringArray.GetString(3));
         }
 
+        [Fact]
+        public void TestStructArray()
+        {
+            // The following can be improved with a Builder class for StructArray.
+            List<Field> fields = new List<Field>();
+            Field.Builder fieldBuilder = new Field.Builder();
+            fields.Add(fieldBuilder.Name("Strings").DataType(StringType.Default).Nullable(true).Build());
+            fieldBuilder = new Field.Builder();
+            fields.Add(fieldBuilder.Name("Ints").DataType(Int32Type.Default).Nullable(true).Build());
+            StructType structType = new StructType(fields);
+
+            StringArray.Builder stringBuilder = new StringArray.Builder();
+            StringArray stringArray = stringBuilder.Append("joe").AppendNull().AppendNull().Append("mark").Build();
+            Int32Array.Builder intBuilder = new Int32Array.Builder();
+            Int32Array intArray = intBuilder.Append(1).Append(2).AppendNull().Append(4).Build();
+            List<Array> arrays = new List<Array>();
+            arrays.Add(stringArray);
+            arrays.Add(intArray);
+
+            ArrowBuffer.BitmapBuilder nullBitmap = new ArrowBuffer.BitmapBuilder();
+            var nullBitmapBuffer = nullBitmap.Append(true).Append(true).Append(false).Append(true).Build();
+            StructArray structs = new StructArray(structType, 4, arrays, nullBitmapBuffer, 1);
+
+            Assert.Equal(4, structs.Length);

Review comment:
       Replace the following asserts with ArrayComparer




----------------------------------------------------------------
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 #8348: ARROW-6972: [C#] Support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Ipc/MessageSerializer.cs
##########
@@ -57,16 +58,30 @@ internal static Schema GetSchema(Flatbuf.Schema schema)
             for (int i = 0; i < schema.FieldsLength; i++)
             {
                 Flatbuf.Field field = schema.Fields(i).GetValueOrDefault();
-
-                schemaBuilder.Field(
-                    new Field(field.Name, GetFieldArrowType(field), field.Nullable));
+                Field arrowField = FieldFromFlatbuffer(field);
+                schemaBuilder.Field(arrowField);
             }
 
             return schemaBuilder.Build();
         }
 
+        private static Field FieldFromFlatbuffer(Flatbuf.Field flatbufField)
+        {
+            System.Collections.Generic.List<Field> childFields = null;
+            if (flatbufField.ChildrenLength > 0)
+            {
+                childFields = new System.Collections.Generic.List<Field>();

Review comment:
       It wouldn’t need to make a List in the constructor since the ctor now takes an IReadOnlyList. It could just hang on to that.




----------------------------------------------------------------
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] pgovind commented on a change in pull request #8348: ARROW-6972: [C#] Support for StructArrays

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



##########
File path: csharp/test/Apache.Arrow.Tests/StructArrayTests.cs
##########
@@ -0,0 +1,86 @@
+// 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;
+using System.Collections.Generic;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class StructArrayTests
+    {
+        [Fact]
+        public void TestStructArray()
+        {
+            // The following can be improved with a Builder class for StructArray.
+            List<Field> fields = new List<Field>();
+            Field.Builder fieldBuilder = new Field.Builder();
+            fields.Add(fieldBuilder.Name("Strings").DataType(StringType.Default).Nullable(true).Build());
+            fieldBuilder = new Field.Builder();
+            fields.Add(fieldBuilder.Name("Ints").DataType(Int32Type.Default).Nullable(true).Build());
+            StructType structType = new StructType(fields);
+
+            StringArray.Builder stringBuilder = new StringArray.Builder();
+            StringArray stringArray = stringBuilder.Append("joe").AppendNull().AppendNull().Append("mark").Build();
+            Int32Array.Builder intBuilder = new Int32Array.Builder();
+            Int32Array intArray = intBuilder.Append(1).Append(2).AppendNull().Append(4).Build();
+            List<Array> arrays = new List<Array>();
+            arrays.Add(stringArray);
+            arrays.Add(intArray);
+
+            ArrowBuffer.BitmapBuilder nullBitmap = new ArrowBuffer.BitmapBuilder();
+            var nullBitmapBuffer = nullBitmap.Append(true).Append(true).Append(false).Append(true).Build();
+            StructArray structs = new StructArray(structType, 4, arrays, nullBitmapBuffer, 1);
+
+            Assert.Equal(4, structs.Length);
+            Assert.Equal(1, structs.NullCount);
+            ArrayData[] childArrays = structs.Data.Children; // Data for StringArray and Int32Array
+            Assert.Equal(2, childArrays.Length);
+            for (int j = 0; j < childArrays.Length; j++)

Review comment:
       :D Artifact of refactoring. Fixed 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] eerhardt commented on pull request #8348: ARROW-6972: [C#] Support for StructArrays

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


   @kou @nealrichardson - do you know why the assign-rust-labels job failed?
   
   ```
   1s
   Run actions/labeler@2.2.0
     with:
       repo-token: ***
       configuration-path: .github/workflows/dev_labeler/labeler.yml
       sync-labels: true
   Error: TypeError: glob pattern string required
   Error: glob pattern string required
   ```


----------------------------------------------------------------
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 #8348: ARROW-6972: [C#] Support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Arrays/StructArray.cs
##########
@@ -21,13 +21,13 @@ namespace Apache.Arrow
 {
     public class StructArray : Array
     {
-        private readonly List<Array> _fields;
+        private readonly IEnumerable<IArrowArray> _fields;

Review comment:
       I now fill out the `_fields` data correctly.

##########
File path: csharp/src/Apache.Arrow/Arrays/StructArray.cs
##########
@@ -21,13 +21,13 @@ namespace Apache.Arrow
 {
     public class StructArray : Array
     {
-        private readonly List<Array> _fields;
+        private readonly IEnumerable<IArrowArray> _fields;

Review comment:
       It now fills out the `_fields` data correctly.




----------------------------------------------------------------
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 #8348: ARROW-6972: [C#] Support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Types/StructType.cs
##########
@@ -28,7 +28,7 @@ public sealed class StructType : ArrowType
 
         public IEnumerable<Field> Fields => _fields;

Review comment:
       I resolved this by moving `Fields` to the base `NestedType` and obsoleting `Children`.




----------------------------------------------------------------
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] pgovind commented on a change in pull request #8348: ARROW-6972: [C#] Support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Types/StructType.cs
##########
@@ -28,7 +28,7 @@ public sealed class StructType : ArrowType
 
         public IEnumerable<Field> Fields => _fields;

Review comment:
       Removed `_fields`. I'll open another PR to rename all instances of Children with Field (there are 80+ hits, so I'd rather not pollute this PR).




----------------------------------------------------------------
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 #8348: ARROW-6972: [C#] Support for StructArrays

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


   


----------------------------------------------------------------
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 #8348: ARROW-6972: [C#] Support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowStreamWriter.cs
##########
@@ -102,6 +103,31 @@ public void Visit(BinaryArray array)
                 _buffers.Add(CreateBuffer(array.ValueBuffer));
             }
 
+            private void Visit(ArrayData[] children)
+            {
+                for (int i = 0; i < children.Length; i++)
+                {
+                    Visit(ArrowArrayFactory.BuildArray(children[i]));

Review comment:
       This is now fixed with the latest 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] pgovind commented on a change in pull request #8348: ARROW-6972: [C#] Support for StructArrays

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



##########
File path: csharp/src/Apache.Arrow/Ipc/MessageSerializer.cs
##########
@@ -57,16 +58,30 @@ internal static Schema GetSchema(Flatbuf.Schema schema)
             for (int i = 0; i < schema.FieldsLength; i++)
             {
                 Flatbuf.Field field = schema.Fields(i).GetValueOrDefault();
-
-                schemaBuilder.Field(
-                    new Field(field.Name, GetFieldArrowType(field), field.Nullable));
+                Field arrowField = FieldFromFlatbuffer(field);
+                schemaBuilder.Field(arrowField);
             }
 
             return schemaBuilder.Build();
         }
 
+        private static Field FieldFromFlatbuffer(Flatbuf.Field flatbufField)
+        {
+            System.Collections.Generic.List<Field> childFields = null;
+            if (flatbufField.ChildrenLength > 0)
+            {
+                childFields = new System.Collections.Generic.List<Field>();

Review comment:
       The `StructType` constructor (really the base `NestedType` constructor) requires an `IReadOnlyList`. I did make the change to specify the capacity upfront 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