You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/08/24 18:10:22 UTC

[GitHub] [arrow] HashidaTKS opened a new pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

HashidaTKS opened a new pull request #10990:
URL: https://github.com/apache/arrow/pull/10990


   - Delta dictionaries for the dictionary type is not supported yet
   - Fix a bug of ListArray.Build()
     - This method was adding a needless "true" to the trailing edge of the validity buffer
   - Make ArrowArrayBuilderFactory public so that we can use it in tests


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       Creating `arrays` for a delta dictionary causes needless memory allocation for  `ArrowArray`.
   It's enough to create just `ArrayData` for the delta dictionary.
   I plan to fix it with future PRs that adding support for writing delta dictionaries.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayConcatenatorTests.cs
##########
@@ -0,0 +1,383 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ArrowArrayConcatenatorTests
+    {
+        [Fact]
+        public void TestStandardCases()
+        {
+            foreach ((List<IArrowArray> testTargetArrayList, IArrowArray expectedArray) in GenerateTestData())
+            {
+                IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(testTargetArrayList);
+                ArrowReaderVerifier.CompareArrays(expectedArray, actualArray);
+            }
+        }
+
+        [Fact]
+        public void TestNullOrEmpty()
+        {
+            Assert.Null(ArrowArrayConcatenator.Concatenate(null));
+            Assert.Null(ArrowArrayConcatenator.Concatenate(new List<IArrowArray>()));
+        }
+
+        [Fact]
+        public void TestSingleElement()
+        {
+            Int32Array array = new Int32Array.Builder().Append(1).Append(2).Build();
+            IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(new [] { array });
+            ArrowReaderVerifier.CompareArrays(array, actualArray);
+        }
+
+        private static IEnumerable<Tuple<List<IArrowArray>, IArrowArray>> GenerateTestData()
+        {
+            var targetTypes = new List<IArrowType>() {
+                    BooleanType.Default,
+                    Int8Type.Default,
+                    Int16Type.Default,
+                    Int32Type.Default,
+                    Int64Type.Default,
+                    UInt8Type.Default,
+                    UInt16Type.Default,
+                    UInt32Type.Default,
+                    UInt64Type.Default,
+                    FloatType.Default,
+                    DoubleType.Default,
+                    BinaryType.Default,
+                    StringType.Default,
+                    Date32Type.Default,
+                    Date64Type.Default,
+                    TimestampType.Default,
+                    new Decimal128Type(14, 10),
+                    new Decimal256Type(14,10),
+                    new ListType(Int64Type.Default),
+                    new StructType(new List<Field>{
+                        new Field.Builder().Name("Strings").DataType(StringType.Default).Nullable(true).Build(),
+                        new Field.Builder().Name("Ints").DataType(Int32Type.Default).Nullable(true).Build()
+                    }),
+                };
+
+            foreach (IArrowType type in targetTypes)
+            {
+                var creator = new TestDataGenerator();
+                type.Accept(creator);
+                yield return Tuple.Create(creator.TestTargetArrayList, creator.ExpectedArray);
+            }
+        }
+
+        private class TestDataGenerator :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<Int8Type>,
+            IArrowTypeVisitor<Int16Type>,
+            IArrowTypeVisitor<Int32Type>,
+            IArrowTypeVisitor<Int64Type>,
+            IArrowTypeVisitor<UInt8Type>,
+            IArrowTypeVisitor<UInt16Type>,
+            IArrowTypeVisitor<UInt32Type>,
+            IArrowTypeVisitor<UInt64Type>,
+            IArrowTypeVisitor<FloatType>,
+            IArrowTypeVisitor<DoubleType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<Decimal128Type>,
+            IArrowTypeVisitor<Decimal256Type>,
+            IArrowTypeVisitor<Date32Type>,
+            IArrowTypeVisitor<Date64Type>,
+            IArrowTypeVisitor<TimestampType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+
+            private List<List<int?>> _baseData;
+
+            private int _baseDataListCount;
+
+            private int _baseDataTotalElementCount;
+
+            public List<IArrowArray> TestTargetArrayList { get; }
+            public IArrowArray ExpectedArray { get; private set; }
+
+            public TestDataGenerator()
+            {
+                _baseData = new List<List<int?>> {
+                    new List<int?> { 1, 2, 3 },
+                    new List<int?> { 100, 101, null },
+                    new List<int?> { 11, null, 12 },
+                };
+
+                _baseDataListCount = _baseData.Count;
+                _baseDataTotalElementCount = _baseData.Sum(_ => _.Count);
+                TestTargetArrayList = new List<IArrowArray>(_baseDataListCount);
+            }
+
+            public void Visit(BooleanType type) => GenerateTestData<bool, BooleanArray, BooleanArray.Builder>(type, x => x % 2 == 0);
+            public void Visit(Int8Type type) => GenerateTestData<sbyte, Int8Array, Int8Array.Builder>(type, x => (sbyte)x);
+            public void Visit(Int16Type type) => GenerateTestData<short, Int16Array, Int16Array.Builder>(type, x => (short)x);
+            public void Visit(Int32Type type) => GenerateTestData<int, Int32Array, Int32Array.Builder>(type, x => x);
+            public void Visit(Int64Type type) => GenerateTestData<long, Int64Array, Int64Array.Builder>(type, x => x);
+            public void Visit(UInt8Type type) => GenerateTestData<byte, UInt8Array, UInt8Array.Builder>(type, x => (byte)x);
+            public void Visit(UInt16Type type) => GenerateTestData<ushort, UInt16Array, UInt16Array.Builder>(type, x => (ushort)x);
+            public void Visit(UInt32Type type) => GenerateTestData<uint, UInt32Array, UInt32Array.Builder>(type, x => (uint)x);
+            public void Visit(UInt64Type type) => GenerateTestData<ulong, UInt64Array, UInt64Array.Builder>(type, x => (ulong)x);
+            public void Visit(FloatType type) => GenerateTestData<float, FloatArray, FloatArray.Builder>(type, x => x);
+            public void Visit(DoubleType type) => GenerateTestData<double, DoubleArray, DoubleArray.Builder>(type, x => x);
+            public void Visit(Date32Type type) => GenerateTestData<DateTime, Date32Array, Date32Array.Builder>(type, x => DateTime.MinValue.AddDays(x));
+            public void Visit(Date64Type type) => GenerateTestData<DateTime, Date64Array, Date64Array.Builder>(type, x => DateTime.MinValue.AddDays(x));

Review comment:
       The `TestData` class is dependent on `Length` and unable to generate flexible data for now.
   For example, when a type is `Int32Type`, it can not create data started with a non-zero value like `[2,3,4]`.
   On the other hand, we need to create that kind of data for testing array concatenation.
   E.g. an expected concatenated list is `[0,1,2,3,4,5,6,7,null]` and base lists are `[0,1,2]`, `[3,4,5]`, `[6.7.null]`.
   
   In order to reuse the `TestData` class, we need to modify it to be able to handle arbitrary data, but it might have a considerable impact.
   For the above reasons, I didn't reuse(modify) it.
   
   I think it is better to modify the `TestData` class and use it in `ArrowArrayConcatenatorTests` in future.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       Creating `arrays` for a delta dictionary causes needless memory allocation for  `ArrowArray`.
   It's enough to create just `ArrayData` for the delta dictionary.
   I plan to fix it with future PRs that add support for writing delta dictionaries.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       Creating `arrays` for a delta dictionary causes needless memory allocation for  `ArrowArray`.
   It's enough to create just `ArrayData` for the delta dictionary.
   I plan to fix it with a future 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       Creating `arrays` for a delta dictionary causes needless memory allocation for  `ArrowArray`.
   It's enough to create just `ArrayData` for the delta dictionary.
   I plan to fix it with future PRs that add support for writing delta dictionaries, or another independent 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] eerhardt closed pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] eerhardt closed pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayBuilderFactoryReflector.cs
##########
@@ -0,0 +1,32 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Reflection;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow.Tests
+{
+    static class ArrayArrayBuilderFactoryReflector
+    {
+        private static readonly Type s_arrowArrayBuilderFactoryType = Assembly.Load("Apache.Arrow").GetType("Apache.Arrow.ArrowArrayBuilderFactory");

Review comment:
       It was left for just in case we should call other methods in future, but it was meaningless at least now, so I have removed this type field.

##########
File path: csharp/src/Apache.Arrow/Arrays/ListArray.cs
##########
@@ -77,7 +77,7 @@ public Builder AppendNull()
 
             public ListArray Build(MemoryAllocator allocator = default)
             {
-                Append();
+                ValueOffsetsBufferBuilder.Append(ValueBuilder.Length);

Review comment:
       OK, I have added a test for this.

##########
File path: csharp/src/Apache.Arrow/Arrays/ListArray.cs
##########
@@ -77,7 +77,7 @@ public Builder AppendNull()
 
             public ListArray Build(MemoryAllocator allocator = default)
             {
-                Append();
+                ValueOffsetsBufferBuilder.Append(ValueBuilder.Length);

Review comment:
       OK, I have added a test for this.
   (`ArrowArrayConcatenatorTests` implicitly tests this bug as well.)




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    public static class ArrayDataConcatenator
+    {
+        public static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatinateVisitor = new ArrayDataConcatinationVisitor(arrayDataList, allocator);
+
+            IArrowType type = arrayDataList[0].DataType;
+            type.Accept(arrowArrayConcatinateVisitor);
+
+            return arrowArrayConcatinateVisitor.Result;
+        }
+
+        private class ArrayDataConcatinationVisitor :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<FixedWidthType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+            public ArrayData Result { get; private set; }
+            private IReadOnlyList<ArrayData> _arrayDataList;
+            private readonly int _totalLength;
+            private readonly int _totalNullCount;
+            private readonly MemoryAllocator _allocator;
+
+            public ArrayDataConcatinationVisitor(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+            {
+                _arrayDataList = arrayDataList;
+                _allocator = allocator;
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    _totalLength += arrayData.Length;
+                    _totalNullCount += arrayData.NullCount;
+                }
+            }
+
+            public void Visit(BooleanType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateBitmapBuffer(1);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(FixedWidthType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateFixedWidthTypeValueBuffer(type);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(BinaryType type) => ConcateneteVariableBinaryArrayData(type);
+
+            public void Visit(StringType type) => ConcateneteVariableBinaryArrayData(type);
+
+            public void Visit(ListType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrayData child = Concatenate(SelectChildren(0), _allocator);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer }, new[] { child });
+            }
+
+            public void Visit(StructType type)
+            {
+                CheckData(type, 1);
+                List<ArrayData> children = new List<ArrayData>(type.Fields.Count);
+
+                for (int i = 0; i < type.Fields.Count; i++)
+                {
+                    children.Add(Concatenate(SelectChildren(i), _allocator));
+                }
+
+                Result = new ArrayData(type, _arrayDataList[0].Length, _arrayDataList[0].NullCount, 0, _arrayDataList[0].Buffers, children);
+            }
+
+            public void Visit(IArrowType type)
+            {
+                throw new NotImplementedException($"Concatination for {type.Name} is not supported yet.");
+            }
+
+            private void CheckData(IArrowType type, int expectedBufferCount)
+            {
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    arrayData.EnsureDataType(type.TypeId);
+                    arrayData.EnsureBufferCount(expectedBufferCount);
+                }
+            }
+
+            private void ConcateneteVariableBinaryArrayData(IArrowType type)
+            {
+                CheckData(type, 3);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrowBuffer valueBuffer = ConcatenateVariableBinaryValueBuffer();
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer, valueBuffer });
+            }
+
+            private ArrowBuffer ConcatenateValidityBuffer()
+            {
+                if (_totalNullCount == 0)
+                {
+                    return ArrowBuffer.Empty;
+                }
+
+                return ConcatenateBitmapBuffer(0);
+            }
+
+            private ArrowBuffer ConcatenateBitmapBuffer(int bufferIndex)
+            {
+                var builder = new ArrowBuffer.BitmapBuilder(_totalLength);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int length = arrayData.Length;
+                    ReadOnlySpan<byte> span = arrayData.Buffers[bufferIndex].Span;
+
+                    for (int i = 0; i < length; i++)
+                    {
+                        builder.Append(span.IsEmpty || BitUtility.GetBit(span, i));
+                    }
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcatenateFixedWidthTypeValueBuffer(FixedWidthType type)
+            {
+                int typeByteWidth = type.BitWidth / 8;
+                var builder = new ArrowBuffer.Builder<byte>(_totalLength * typeByteWidth);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int length = arrayData.Length;
+                    int byteLength = length * typeByteWidth;
+
+                    builder.Append(arrayData.Buffers[1].Span.Slice(0, byteLength));
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcatenateVariableBinaryValueBuffer()
+            {
+                var builder = new ArrowBuffer.Builder<byte>();
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int lastOffset = arrayData.Buffers[1].Span.CastTo<int>()[arrayData.Length];
+                    builder.Append(arrayData.Buffers[2].Span.Slice(0, lastOffset));
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcateneteOffsetBuffer()
+            {
+                var builder = new ArrowBuffer.Builder<int>(_totalLength + 1);
+                int baseOffset = 0;
+
+                builder.Append(0);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    if (arrayData.Length == 0)
+                    {
+                        continue;
+                    }
+
+                    // The first offset is always 0.
+                    // It should be skipped because it duplicate to the last offset of builder.
+                    ReadOnlySpan<int> span = arrayData.Buffers[1].Span.CastTo<int>().Slice(1, arrayData.Length);

Review comment:
       It is a bit confusing, but the length of `arrayData.Buffers[1]` is `arrayData.Length + 1`, so I think being `arrayData.Length` is correct.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       Creating `arrays` for a delta dictionary causes needless memory allocation for  `ArrowArray`.
   It's enough to create just `ArrayData` for the delta dictionary.
   I plan to fix them in future PRs.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Arrays/ArrowArrayBuilderFactory.cs
##########
@@ -18,9 +18,9 @@
 
 namespace Apache.Arrow
 {
-    static class ArrowArrayBuilderFactory
+    public static class ArrowArrayBuilderFactory

Review comment:
       I have changed them to `internal`.

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrowArrayBuilderFactory.cs
##########
@@ -18,9 +18,9 @@
 
 namespace Apache.Arrow
 {
-    static class ArrowArrayBuilderFactory
+    public static class ArrowArrayBuilderFactory

Review comment:
       I have changed it to `internal`.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   @eerhardt 
   Could you please review this when you have time?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] eerhardt commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   @HashidaTKS - have you pushed the changes? I don't see any recent 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayConcatenatorTests.cs
##########
@@ -0,0 +1,383 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ArrowArrayConcatenatorTests
+    {
+        [Fact]
+        public void TestStandardCases()
+        {
+            foreach ((List<IArrowArray> testTargetArrayList, IArrowArray expectedArray) in GenerateTestData())
+            {
+                IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(testTargetArrayList);
+                ArrowReaderVerifier.CompareArrays(expectedArray, actualArray);
+            }
+        }
+
+        [Fact]
+        public void TestNullOrEmpty()
+        {
+            Assert.Null(ArrowArrayConcatenator.Concatenate(null));
+            Assert.Null(ArrowArrayConcatenator.Concatenate(new List<IArrowArray>()));
+        }
+
+        [Fact]
+        public void TestSingleElement()
+        {
+            Int32Array array = new Int32Array.Builder().Append(1).Append(2).Build();
+            IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(new [] { array });
+            ArrowReaderVerifier.CompareArrays(array, actualArray);
+        }
+
+        private static IEnumerable<Tuple<List<IArrowArray>, IArrowArray>> GenerateTestData()
+        {
+            var targetTypes = new List<IArrowType>() {
+                    BooleanType.Default,
+                    Int8Type.Default,
+                    Int16Type.Default,
+                    Int32Type.Default,
+                    Int64Type.Default,
+                    UInt8Type.Default,
+                    UInt16Type.Default,
+                    UInt32Type.Default,
+                    UInt64Type.Default,
+                    FloatType.Default,
+                    DoubleType.Default,
+                    BinaryType.Default,
+                    StringType.Default,
+                    Date32Type.Default,
+                    Date64Type.Default,
+                    TimestampType.Default,
+                    new Decimal128Type(14, 10),
+                    new Decimal256Type(14,10),
+                    new ListType(Int64Type.Default),
+                    new StructType(new List<Field>{
+                        new Field.Builder().Name("Strings").DataType(StringType.Default).Nullable(true).Build(),
+                        new Field.Builder().Name("Ints").DataType(Int32Type.Default).Nullable(true).Build()
+                    }),
+                };
+
+            foreach (IArrowType type in targetTypes)
+            {
+                var creator = new TestDataGenerator();
+                type.Accept(creator);
+                yield return Tuple.Create(creator.TestTargetArrayList, creator.ExpectedArray);
+            }
+        }
+
+        private class TestDataGenerator :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<Int8Type>,
+            IArrowTypeVisitor<Int16Type>,
+            IArrowTypeVisitor<Int32Type>,
+            IArrowTypeVisitor<Int64Type>,
+            IArrowTypeVisitor<UInt8Type>,
+            IArrowTypeVisitor<UInt16Type>,
+            IArrowTypeVisitor<UInt32Type>,
+            IArrowTypeVisitor<UInt64Type>,
+            IArrowTypeVisitor<FloatType>,
+            IArrowTypeVisitor<DoubleType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<Decimal128Type>,
+            IArrowTypeVisitor<Decimal256Type>,
+            IArrowTypeVisitor<Date32Type>,
+            IArrowTypeVisitor<Date64Type>,
+            IArrowTypeVisitor<TimestampType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+
+            private List<List<int?>> _baseData;
+
+            private int _baseDataListCount;
+
+            private int _baseDataTotalElementCount;
+
+            public List<IArrowArray> TestTargetArrayList { get; }
+            public IArrowArray ExpectedArray { get; private set; }
+
+            public TestDataGenerator()
+            {
+                _baseData = new List<List<int?>> {
+                    new List<int?> { 1, 2, 3 },
+                    new List<int?> { 100, 101, null },
+                    new List<int?> { 11, null, 12 },
+                };
+
+                _baseDataListCount = _baseData.Count;
+                _baseDataTotalElementCount = _baseData.Sum(_ => _.Count);
+                TestTargetArrayList = new List<IArrowArray>(_baseDataListCount);
+            }
+
+            public void Visit(BooleanType type) => GenerateTestData<bool, BooleanArray, BooleanArray.Builder>(type, x => x % 2 == 0);
+            public void Visit(Int8Type type) => GenerateTestData<sbyte, Int8Array, Int8Array.Builder>(type, x => (sbyte)x);
+            public void Visit(Int16Type type) => GenerateTestData<short, Int16Array, Int16Array.Builder>(type, x => (short)x);
+            public void Visit(Int32Type type) => GenerateTestData<int, Int32Array, Int32Array.Builder>(type, x => x);
+            public void Visit(Int64Type type) => GenerateTestData<long, Int64Array, Int64Array.Builder>(type, x => x);
+            public void Visit(UInt8Type type) => GenerateTestData<byte, UInt8Array, UInt8Array.Builder>(type, x => (byte)x);
+            public void Visit(UInt16Type type) => GenerateTestData<ushort, UInt16Array, UInt16Array.Builder>(type, x => (ushort)x);
+            public void Visit(UInt32Type type) => GenerateTestData<uint, UInt32Array, UInt32Array.Builder>(type, x => (uint)x);
+            public void Visit(UInt64Type type) => GenerateTestData<ulong, UInt64Array, UInt64Array.Builder>(type, x => (ulong)x);
+            public void Visit(FloatType type) => GenerateTestData<float, FloatArray, FloatArray.Builder>(type, x => x);
+            public void Visit(DoubleType type) => GenerateTestData<double, DoubleArray, DoubleArray.Builder>(type, x => x);
+            public void Visit(Date32Type type) => GenerateTestData<DateTime, Date32Array, Date32Array.Builder>(type, x => DateTime.MinValue.AddDays(x));
+            public void Visit(Date64Type type) => GenerateTestData<DateTime, Date64Array, Date64Array.Builder>(type, x => DateTime.MinValue.AddDays(x));

Review comment:
       The `TestData` class is dependent on `Length` and unable to generate flexible data for now.
   For example, when a type is `Int32Type`, it can not create data starts with a non-zero value like `[3,4,5]`.
   On the other hand, we need to create that kind of data for testing array concatenation.
   E.g. an expected concatenated array is `[0,1,2,3,4,5,6,7,null]` and base arrays are `[0,1,2]`, `[3,4,5]`, `[6.7.null]`.
   
   In order to reuse the `TestData` class, we need to modify it to be able to handle arbitrary data, but it might have a considerable impact.
   For the above reasons, I didn't reuse(modify) it.
   
   I think it is better to modify the `TestData` class and use it in `ArrowArrayConcatenatorTests` in future.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] eerhardt commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    public static class ArrayDataConcatenator

Review comment:
       Is it possible to mark the new classes as `internal`? Making them public means that external callers can call them, and we then need to maintain the public surface area (unless we want to make breaking changes).
   
   If this is only marked `public` for testing, instead, try as hard as you can to write tests using the `public` API - like a customer would.
   
   If that can't be done (since we can't write dictionaries in C# yet), consider either using InternalsVisibleTo or Reflection (preferred) to call these methods from the tests.

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrowArrayBuilderFactory.cs
##########
@@ -18,9 +18,9 @@
 
 namespace Apache.Arrow
 {
-    static class ArrowArrayBuilderFactory
+    public static class ArrowArrayBuilderFactory

Review comment:
       Is it possible to not make this public API? Would normal callers need it? We should avoid making it public API just for testing.

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    public static class ArrayDataConcatenator
+    {
+        public static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatinateVisitor = new ArrayDataConcatinationVisitor(arrayDataList, allocator);
+
+            IArrowType type = arrayDataList[0].DataType;
+            type.Accept(arrowArrayConcatinateVisitor);
+
+            return arrowArrayConcatinateVisitor.Result;
+        }
+
+        private class ArrayDataConcatinationVisitor :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<FixedWidthType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+            public ArrayData Result { get; private set; }
+            private IReadOnlyList<ArrayData> _arrayDataList;

Review comment:
       ```suggestion
               private readonly IReadOnlyList<ArrayData> _arrayDataList;
   ```

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    public static class ArrayDataConcatenator
+    {
+        public static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatinateVisitor = new ArrayDataConcatinationVisitor(arrayDataList, allocator);
+
+            IArrowType type = arrayDataList[0].DataType;
+            type.Accept(arrowArrayConcatinateVisitor);
+
+            return arrowArrayConcatinateVisitor.Result;
+        }
+
+        private class ArrayDataConcatinationVisitor :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<FixedWidthType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+            public ArrayData Result { get; private set; }
+            private IReadOnlyList<ArrayData> _arrayDataList;
+            private readonly int _totalLength;
+            private readonly int _totalNullCount;
+            private readonly MemoryAllocator _allocator;
+
+            public ArrayDataConcatinationVisitor(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+            {
+                _arrayDataList = arrayDataList;
+                _allocator = allocator;
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    _totalLength += arrayData.Length;
+                    _totalNullCount += arrayData.NullCount;
+                }
+            }
+
+            public void Visit(BooleanType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateBitmapBuffer(1);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(FixedWidthType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateFixedWidthTypeValueBuffer(type);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(BinaryType type) => ConcateneteVariableBinaryArrayData(type);
+
+            public void Visit(StringType type) => ConcateneteVariableBinaryArrayData(type);
+
+            public void Visit(ListType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrayData child = Concatenate(SelectChildren(0), _allocator);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer }, new[] { child });
+            }
+
+            public void Visit(StructType type)
+            {
+                CheckData(type, 1);
+                List<ArrayData> children = new List<ArrayData>(type.Fields.Count);
+
+                for (int i = 0; i < type.Fields.Count; i++)
+                {
+                    children.Add(Concatenate(SelectChildren(i), _allocator));
+                }
+
+                Result = new ArrayData(type, _arrayDataList[0].Length, _arrayDataList[0].NullCount, 0, _arrayDataList[0].Buffers, children);
+            }
+
+            public void Visit(IArrowType type)
+            {
+                throw new NotImplementedException($"Concatination for {type.Name} is not supported yet.");
+            }
+
+            private void CheckData(IArrowType type, int expectedBufferCount)
+            {
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    arrayData.EnsureDataType(type.TypeId);
+                    arrayData.EnsureBufferCount(expectedBufferCount);
+                }
+            }
+
+            private void ConcateneteVariableBinaryArrayData(IArrowType type)
+            {
+                CheckData(type, 3);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrowBuffer valueBuffer = ConcatenateVariableBinaryValueBuffer();
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer, valueBuffer });
+            }
+
+            private ArrowBuffer ConcatenateValidityBuffer()
+            {
+                if (_totalNullCount == 0)
+                {
+                    return ArrowBuffer.Empty;
+                }
+
+                return ConcatenateBitmapBuffer(0);
+            }
+
+            private ArrowBuffer ConcatenateBitmapBuffer(int bufferIndex)
+            {
+                var builder = new ArrowBuffer.BitmapBuilder(_totalLength);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int length = arrayData.Length;
+                    ReadOnlySpan<byte> span = arrayData.Buffers[bufferIndex].Span;
+
+                    for (int i = 0; i < length; i++)
+                    {
+                        builder.Append(span.IsEmpty || BitUtility.GetBit(span, i));
+                    }
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcatenateFixedWidthTypeValueBuffer(FixedWidthType type)
+            {
+                int typeByteWidth = type.BitWidth / 8;
+                var builder = new ArrowBuffer.Builder<byte>(_totalLength * typeByteWidth);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int length = arrayData.Length;
+                    int byteLength = length * typeByteWidth;
+
+                    builder.Append(arrayData.Buffers[1].Span.Slice(0, byteLength));
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcatenateVariableBinaryValueBuffer()
+            {
+                var builder = new ArrowBuffer.Builder<byte>();
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int lastOffset = arrayData.Buffers[1].Span.CastTo<int>()[arrayData.Length];
+                    builder.Append(arrayData.Buffers[2].Span.Slice(0, lastOffset));
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcateneteOffsetBuffer()
+            {
+                var builder = new ArrowBuffer.Builder<int>(_totalLength + 1);
+                int baseOffset = 0;
+
+                builder.Append(0);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    if (arrayData.Length == 0)
+                    {
+                        continue;
+                    }
+
+                    // The first offset is always 0.
+                    // It should be skipped because it duplicate to the last offset of builder.
+                    ReadOnlySpan<int> span = arrayData.Buffers[1].Span.CastTo<int>().Slice(1, arrayData.Length);

Review comment:
       Doesn't this need to be `arrayData.Length - 1`? Because you sliced off the first element.
   ```suggestion
                       ReadOnlySpan<int> span = arrayData.Buffers[1].Span.CastTo<int>().Slice(1, arrayData.Length - 1);
   ```

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    public static class ArrayDataConcatenator
+    {
+        public static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatinateVisitor = new ArrayDataConcatinationVisitor(arrayDataList, allocator);
+
+            IArrowType type = arrayDataList[0].DataType;
+            type.Accept(arrowArrayConcatinateVisitor);
+
+            return arrowArrayConcatinateVisitor.Result;
+        }
+
+        private class ArrayDataConcatinationVisitor :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<FixedWidthType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+            public ArrayData Result { get; private set; }
+            private IReadOnlyList<ArrayData> _arrayDataList;
+            private readonly int _totalLength;
+            private readonly int _totalNullCount;
+            private readonly MemoryAllocator _allocator;
+
+            public ArrayDataConcatinationVisitor(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+            {
+                _arrayDataList = arrayDataList;
+                _allocator = allocator;
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    _totalLength += arrayData.Length;
+                    _totalNullCount += arrayData.NullCount;
+                }
+            }
+
+            public void Visit(BooleanType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateBitmapBuffer(1);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(FixedWidthType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateFixedWidthTypeValueBuffer(type);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(BinaryType type) => ConcateneteVariableBinaryArrayData(type);
+
+            public void Visit(StringType type) => ConcateneteVariableBinaryArrayData(type);
+
+            public void Visit(ListType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrayData child = Concatenate(SelectChildren(0), _allocator);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer }, new[] { child });
+            }
+
+            public void Visit(StructType type)
+            {
+                CheckData(type, 1);
+                List<ArrayData> children = new List<ArrayData>(type.Fields.Count);
+
+                for (int i = 0; i < type.Fields.Count; i++)
+                {
+                    children.Add(Concatenate(SelectChildren(i), _allocator));
+                }
+
+                Result = new ArrayData(type, _arrayDataList[0].Length, _arrayDataList[0].NullCount, 0, _arrayDataList[0].Buffers, children);
+            }
+
+            public void Visit(IArrowType type)
+            {
+                throw new NotImplementedException($"Concatination for {type.Name} is not supported yet.");
+            }
+
+            private void CheckData(IArrowType type, int expectedBufferCount)
+            {
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    arrayData.EnsureDataType(type.TypeId);
+                    arrayData.EnsureBufferCount(expectedBufferCount);
+                }
+            }
+
+            private void ConcateneteVariableBinaryArrayData(IArrowType type)

Review comment:
       ```suggestion
               private void ConcatenateVariableBinaryArrayData(IArrowType type)
   ```

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    public static class ArrayDataConcatenator
+    {
+        public static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatinateVisitor = new ArrayDataConcatinationVisitor(arrayDataList, allocator);

Review comment:
       ```suggestion
               var arrowArrayConcatenationVisitor = new ArrayDataConcatenationVisitor(arrayDataList, allocator);
   ```

##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayConcatenatorTests.cs
##########
@@ -0,0 +1,383 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ArrowArrayConcatenatorTests
+    {
+        [Fact]
+        public void TestStandardCases()
+        {
+            foreach ((List<IArrowArray> testTargetArrayList, IArrowArray expectedArray) in GenerateTestData())
+            {
+                IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(testTargetArrayList);
+                ArrowReaderVerifier.CompareArrays(expectedArray, actualArray);
+            }
+        }
+
+        [Fact]
+        public void TestNullOrEmpty()
+        {
+            Assert.Null(ArrowArrayConcatenator.Concatenate(null));
+            Assert.Null(ArrowArrayConcatenator.Concatenate(new List<IArrowArray>()));
+        }
+
+        [Fact]
+        public void TestSingleElement()
+        {
+            Int32Array array = new Int32Array.Builder().Append(1).Append(2).Build();
+            IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(new [] { array });
+            ArrowReaderVerifier.CompareArrays(array, actualArray);
+        }
+
+        private static IEnumerable<Tuple<List<IArrowArray>, IArrowArray>> GenerateTestData()
+        {
+            var targetTypes = new List<IArrowType>() {
+                    BooleanType.Default,
+                    Int8Type.Default,
+                    Int16Type.Default,
+                    Int32Type.Default,
+                    Int64Type.Default,
+                    UInt8Type.Default,
+                    UInt16Type.Default,
+                    UInt32Type.Default,
+                    UInt64Type.Default,
+                    FloatType.Default,
+                    DoubleType.Default,
+                    BinaryType.Default,
+                    StringType.Default,
+                    Date32Type.Default,
+                    Date64Type.Default,
+                    TimestampType.Default,
+                    new Decimal128Type(14, 10),
+                    new Decimal256Type(14,10),
+                    new ListType(Int64Type.Default),
+                    new StructType(new List<Field>{
+                        new Field.Builder().Name("Strings").DataType(StringType.Default).Nullable(true).Build(),
+                        new Field.Builder().Name("Ints").DataType(Int32Type.Default).Nullable(true).Build()
+                    }),
+                };
+
+            foreach (IArrowType type in targetTypes)
+            {
+                var creator = new TestDataGenerator();
+                type.Accept(creator);
+                yield return Tuple.Create(creator.TestTargetArrayList, creator.ExpectedArray);
+            }
+        }
+
+        private class TestDataGenerator :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<Int8Type>,
+            IArrowTypeVisitor<Int16Type>,
+            IArrowTypeVisitor<Int32Type>,
+            IArrowTypeVisitor<Int64Type>,
+            IArrowTypeVisitor<UInt8Type>,
+            IArrowTypeVisitor<UInt16Type>,
+            IArrowTypeVisitor<UInt32Type>,
+            IArrowTypeVisitor<UInt64Type>,
+            IArrowTypeVisitor<FloatType>,
+            IArrowTypeVisitor<DoubleType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<Decimal128Type>,
+            IArrowTypeVisitor<Decimal256Type>,
+            IArrowTypeVisitor<Date32Type>,
+            IArrowTypeVisitor<Date64Type>,
+            IArrowTypeVisitor<TimestampType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+
+            private List<List<int?>> _baseData;
+
+            private int _baseDataListCount;
+
+            private int _baseDataTotalElementCount;
+
+            public List<IArrowArray> TestTargetArrayList { get; }
+            public IArrowArray ExpectedArray { get; private set; }
+
+            public TestDataGenerator()
+            {
+                _baseData = new List<List<int?>> {
+                    new List<int?> { 1, 2, 3 },
+                    new List<int?> { 100, 101, null },
+                    new List<int?> { 11, null, 12 },
+                };
+
+                _baseDataListCount = _baseData.Count;
+                _baseDataTotalElementCount = _baseData.Sum(_ => _.Count);
+                TestTargetArrayList = new List<IArrowArray>(_baseDataListCount);
+            }
+
+            public void Visit(BooleanType type) => GenerateTestData<bool, BooleanArray, BooleanArray.Builder>(type, x => x % 2 == 0);
+            public void Visit(Int8Type type) => GenerateTestData<sbyte, Int8Array, Int8Array.Builder>(type, x => (sbyte)x);
+            public void Visit(Int16Type type) => GenerateTestData<short, Int16Array, Int16Array.Builder>(type, x => (short)x);
+            public void Visit(Int32Type type) => GenerateTestData<int, Int32Array, Int32Array.Builder>(type, x => x);
+            public void Visit(Int64Type type) => GenerateTestData<long, Int64Array, Int64Array.Builder>(type, x => x);
+            public void Visit(UInt8Type type) => GenerateTestData<byte, UInt8Array, UInt8Array.Builder>(type, x => (byte)x);
+            public void Visit(UInt16Type type) => GenerateTestData<ushort, UInt16Array, UInt16Array.Builder>(type, x => (ushort)x);
+            public void Visit(UInt32Type type) => GenerateTestData<uint, UInt32Array, UInt32Array.Builder>(type, x => (uint)x);
+            public void Visit(UInt64Type type) => GenerateTestData<ulong, UInt64Array, UInt64Array.Builder>(type, x => (ulong)x);
+            public void Visit(FloatType type) => GenerateTestData<float, FloatArray, FloatArray.Builder>(type, x => x);
+            public void Visit(DoubleType type) => GenerateTestData<double, DoubleArray, DoubleArray.Builder>(type, x => x);
+            public void Visit(Date32Type type) => GenerateTestData<DateTime, Date32Array, Date32Array.Builder>(type, x => DateTime.MinValue.AddDays(x));
+            public void Visit(Date64Type type) => GenerateTestData<DateTime, Date64Array, Date64Array.Builder>(type, x => DateTime.MinValue.AddDays(x));

Review comment:
       Can we reuse the existing code in the existing `TestData` class?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS edited a comment on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

Posted by GitBox <gi...@apache.org>.
HashidaTKS edited a comment on pull request #10990:
URL: https://github.com/apache/arrow/pull/10990#issuecomment-928209072


   The internal `ListArray.Builder` ctor was called instead of the public ctor in `ArrayBuilderTests.NestedListArrayBuilder()`. 
   Because of adding `InternalsVisibleTo` into `AssemblyInfo.cs`, `Arrow.Array.Test` could see the internal ctor and it had higher overload priority than the public ctor.
   
   ```
   public Builder(IArrowType valueDataType) ...
   internal Builder(ListType dataType) ...
   ```
   
   For the above reasons, I changed to use Reflection instead of InternalsVisibleTo in tests.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS edited a comment on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

Posted by GitBox <gi...@apache.org>.
HashidaTKS edited a comment on pull request #10990:
URL: https://github.com/apache/arrow/pull/10990#issuecomment-928209072


   The internal `ListArray.Builder` ctor was called instead of the public ctor in `ArrayBuilderTests.NestedListArrayBuilder()`. 
   Because of adding `InternalsVisibleTo` into `AssemblyInfo.cs`, `Arrow.Array.Test` could see the internal ctor and it had higher overload priority than the public ctor.
   
   ```
   public Builder(IArrowType valueDataType) ...
   internal Builder(ListType dataType) ...
   ```
   
   For the above reasons, I changed to use Reflection instead of InternalsVisibleTo in tests.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   @eerhardt 
   Thanks, I have addressed your comments.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] eerhardt commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrayDataConcatenatorTests.cs
##########
@@ -0,0 +1,52 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Collections.Generic;
+using System.Reflection;
+using Apache.Arrow.Memory;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ArrayDataConcatenatorTests
+    {
+        [Fact]
+        public void TestNullOrEmpty()
+        {
+            Assert.Null(ArrayDataConcatenatorReflector.InvokeConcatenate(null));
+            Assert.Null(ArrayDataConcatenatorReflector.InvokeConcatenate(new List<ArrayData>()));
+        }
+
+        [Fact]
+        public void TestSingleElement()
+        {
+            Int32Array array = new Int32Array.Builder().Append(1).Append(2).Build();
+            ArrayData actualArray = ArrayDataConcatenatorReflector.InvokeConcatenate(new[] { array.Data });
+            ArrowReaderVerifier.CompareArrays(array, ArrowArrayFactory.BuildArray(actualArray));
+        }
+
+        private static class ArrayDataConcatenatorReflector
+        {
+            private static readonly Type s_arrayDataConcatenatorType = Assembly.Load("Apache.Arrow").GetType("Apache.Arrow.ArrayDataConcatenator");

Review comment:
       ```suggestion
               private static readonly Type s_arrayDataConcatenatorType = typeof(ArrayData).Assembly.GetType("Apache.Arrow.ArrayDataConcatenator");
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] eerhardt commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayBuilderFactoryReflector.cs
##########
@@ -0,0 +1,32 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Reflection;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow.Tests
+{
+    static class ArrayArrayBuilderFactoryReflector
+    {
+        private static readonly Type s_arrowArrayBuilderFactoryType = Assembly.Load("Apache.Arrow").GetType("Apache.Arrow.ArrowArrayBuilderFactory");

Review comment:
       Do we need to hold onto this type statically in a field? It could just be inlined into the MethodInfo call below and the field could be removed.

##########
File path: csharp/src/Apache.Arrow/Arrays/ListArray.cs
##########
@@ -77,7 +77,7 @@ public Builder AppendNull()
 
             public ListArray Build(MemoryAllocator allocator = default)
             {
-                Append();
+                ValueOffsetsBufferBuilder.Append(ValueBuilder.Length);

Review comment:
       Can you write a unit test for this to ensure it stays fixed?

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       Note, once my integration change #10973 is merged, you can enable the dictionary integration tests to make sure dictionaries are working across C# and the other languages.

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    static class ArrayDataConcatenator
+    {
+        internal static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatenationVisitor = new ArrayDataConcatenationVisitor(arrayDataList, allocator);
+
+            IArrowType type = arrayDataList[0].DataType;
+            type.Accept(arrowArrayConcatenationVisitor);
+
+            return arrowArrayConcatenationVisitor.Result;
+        }
+
+        private class ArrayDataConcatenationVisitor :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<FixedWidthType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+            public ArrayData Result { get; private set; }
+            private readonly IReadOnlyList<ArrayData> _arrayDataList;
+            private readonly int _totalLength;
+            private readonly int _totalNullCount;
+            private readonly MemoryAllocator _allocator;
+
+            public ArrayDataConcatenationVisitor(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+            {
+                _arrayDataList = arrayDataList;
+                _allocator = allocator;
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    _totalLength += arrayData.Length;
+                    _totalNullCount += arrayData.NullCount;
+                }
+            }
+
+            public void Visit(BooleanType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateBitmapBuffer(1);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(FixedWidthType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateFixedWidthTypeValueBuffer(type);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(BinaryType type) => ConcatenateVariableBinaryArrayData(type);
+
+            public void Visit(StringType type) => ConcatenateVariableBinaryArrayData(type);
+
+            public void Visit(ListType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrayData child = Concatenate(SelectChildren(0), _allocator);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer }, new[] { child });
+            }
+
+            public void Visit(StructType type)
+            {
+                CheckData(type, 1);
+                List<ArrayData> children = new List<ArrayData>(type.Fields.Count);
+
+                for (int i = 0; i < type.Fields.Count; i++)
+                {
+                    children.Add(Concatenate(SelectChildren(i), _allocator));
+                }
+
+                Result = new ArrayData(type, _arrayDataList[0].Length, _arrayDataList[0].NullCount, 0, _arrayDataList[0].Buffers, children);
+            }
+
+            public void Visit(IArrowType type)
+            {
+                throw new NotImplementedException($"Concatination for {type.Name} is not supported yet.");
+            }
+
+            private void CheckData(IArrowType type, int expectedBufferCount)
+            {
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    arrayData.EnsureDataType(type.TypeId);
+                    arrayData.EnsureBufferCount(expectedBufferCount);
+                }
+            }
+
+            private void ConcatenateVariableBinaryArrayData(IArrowType type)
+            {
+                CheckData(type, 3);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrowBuffer valueBuffer = ConcatenateVariableBinaryValueBuffer();
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer, valueBuffer });
+            }
+
+            private ArrowBuffer ConcatenateValidityBuffer()
+            {
+                if (_totalNullCount == 0)
+                {
+                    return ArrowBuffer.Empty;
+                }
+
+                return ConcatenateBitmapBuffer(0);
+            }
+
+            private ArrowBuffer ConcatenateBitmapBuffer(int bufferIndex)
+            {
+                var builder = new ArrowBuffer.BitmapBuilder(_totalLength);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int length = arrayData.Length;
+                    ReadOnlySpan<byte> span = arrayData.Buffers[bufferIndex].Span;
+
+                    for (int i = 0; i < length; i++)
+                    {
+                        builder.Append(span.IsEmpty || BitUtility.GetBit(span, i));
+                    }
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcatenateFixedWidthTypeValueBuffer(FixedWidthType type)
+            {
+                int typeByteWidth = type.BitWidth / 8;
+                var builder = new ArrowBuffer.Builder<byte>(_totalLength * typeByteWidth);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int length = arrayData.Length;
+                    int byteLength = length * typeByteWidth;
+
+                    builder.Append(arrayData.Buffers[1].Span.Slice(0, byteLength));
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcatenateVariableBinaryValueBuffer()
+            {
+                var builder = new ArrowBuffer.Builder<byte>();
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int lastOffset = arrayData.Buffers[1].Span.CastTo<int>()[arrayData.Length];
+                    builder.Append(arrayData.Buffers[2].Span.Slice(0, lastOffset));
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcateneteOffsetBuffer()

Review comment:
       ```suggestion
               private ArrowBuffer ConcatenateOffsetBuffer()
   ```

##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayBuilderFactoryReflector.cs
##########
@@ -0,0 +1,32 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Reflection;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow.Tests
+{
+    static class ArrayArrayBuilderFactoryReflector
+    {
+        private static readonly Type s_arrowArrayBuilderFactoryType = Assembly.Load("Apache.Arrow").GetType("Apache.Arrow.ArrowArrayBuilderFactory");

Review comment:
       ```suggestion
           private static readonly Type s_arrowArrayBuilderFactoryType = typeof(ArrayData).Assembly.GetType("Apache.Arrow.ArrowArrayBuilderFactory");
   ```

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    static class ArrayDataConcatenator
+    {
+        internal static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatenationVisitor = new ArrayDataConcatenationVisitor(arrayDataList, allocator);
+
+            IArrowType type = arrayDataList[0].DataType;
+            type.Accept(arrowArrayConcatenationVisitor);
+
+            return arrowArrayConcatenationVisitor.Result;
+        }
+
+        private class ArrayDataConcatenationVisitor :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<FixedWidthType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+            public ArrayData Result { get; private set; }
+            private readonly IReadOnlyList<ArrayData> _arrayDataList;
+            private readonly int _totalLength;
+            private readonly int _totalNullCount;
+            private readonly MemoryAllocator _allocator;
+
+            public ArrayDataConcatenationVisitor(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+            {
+                _arrayDataList = arrayDataList;
+                _allocator = allocator;
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    _totalLength += arrayData.Length;
+                    _totalNullCount += arrayData.NullCount;
+                }
+            }
+
+            public void Visit(BooleanType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateBitmapBuffer(1);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(FixedWidthType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateFixedWidthTypeValueBuffer(type);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(BinaryType type) => ConcatenateVariableBinaryArrayData(type);
+
+            public void Visit(StringType type) => ConcatenateVariableBinaryArrayData(type);
+
+            public void Visit(ListType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrayData child = Concatenate(SelectChildren(0), _allocator);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer }, new[] { child });
+            }
+
+            public void Visit(StructType type)
+            {
+                CheckData(type, 1);
+                List<ArrayData> children = new List<ArrayData>(type.Fields.Count);
+
+                for (int i = 0; i < type.Fields.Count; i++)
+                {
+                    children.Add(Concatenate(SelectChildren(i), _allocator));
+                }
+
+                Result = new ArrayData(type, _arrayDataList[0].Length, _arrayDataList[0].NullCount, 0, _arrayDataList[0].Buffers, children);
+            }
+
+            public void Visit(IArrowType type)
+            {
+                throw new NotImplementedException($"Concatination for {type.Name} is not supported yet.");

Review comment:
       ```suggestion
                   throw new NotImplementedException($"Concatenation for {type.Name} is not supported yet.");
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   @eerhardt 
   Thanks, I have addressed your comments.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       Creating `arrays` for a delta dictionary causes needless memory allocation for  `ArrowArray`.
   It's enough to create just `ArrayData` for the delta dictionary.
   I plan to fix it with future PRs that add support for writing delta dictionaries, or independent 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] github-actions[bot] commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


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


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   I am checking the failed test. Please wait a while...


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       I tested reading delta dictionaries by creating a test file `delta_dictionary_batch_test.batch` with python and reading `delta_dictionary_batch_test.batch` with C#, and it worked fine.
   
   ```
   import pyarrow as pa
   
   ty = pa.dictionary(pa.int8(), pa.utf8())
   data = [["foo", "foo", None],
               ["foo", "bar", "foo"],
               ["foo", "bar"],
               ["foo", None, "bar", "quux"], 
               ["bar", "quux"],
               ]
   batches = [
           pa.RecordBatch.from_arrays([pa.array(v, type=ty)], names=['dicts'])
           for v in data]
   schema = batches[0].schema
   
   def write_batches():
       options = pa.ipc.IpcWriteOptions(emit_dictionary_deltas=True)
       with pa.RecordBatchStreamWriter("./delta_dictionary_batch_test.batch", schema = schema, options = options) as writer:
           for batch in batches:
               writer.write_batch(batch)
   
   st = write_batches()
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] eerhardt commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   Hey @HashidaTKS - sorry it's taken so long. I plan on getting to this today or tomorrow. Thanks for the patience.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       I tested reading delta dictionaries by creating a test file `delta_dictionary_batch_test.batch` with python and reading `delta_dictionary_batch_test.batch` with C#, and it worked fine.
   
   ```
   import pyarrow as pa
   
   ty = pa.dictionary(pa.int8(), pa.utf8())
   data = [["foo", "foo", None],
               ["foo", "bar", "foo"],
               ["foo", "bar"],
               ["foo", None, "bar", "quux"], 
               ["bar", "quux"],
               ]
   batches = [
           pa.RecordBatch.from_arrays([pa.array(v, type=ty)], names=['dicts'])
           for v in data]
   schema = batches[0].schema
   
   def write_batches():
       options = pa.ipc.IpcWriteOptions(emit_dictionary_deltas=True)
       with pa.RecordBatchStreamWriter("./delta_dictionary_batch_test.batch", schema = schema, options = options) as writer:
           for batch in batches:
               writer.write_batch(batch)
   
   st = write_batches()
   
   import pyarrow as pa
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayConcatenatorTests.cs
##########
@@ -0,0 +1,383 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ArrowArrayConcatenatorTests
+    {
+        [Fact]
+        public void TestStandardCases()
+        {
+            foreach ((List<IArrowArray> testTargetArrayList, IArrowArray expectedArray) in GenerateTestData())
+            {
+                IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(testTargetArrayList);
+                ArrowReaderVerifier.CompareArrays(expectedArray, actualArray);
+            }
+        }
+
+        [Fact]
+        public void TestNullOrEmpty()
+        {
+            Assert.Null(ArrowArrayConcatenator.Concatenate(null));
+            Assert.Null(ArrowArrayConcatenator.Concatenate(new List<IArrowArray>()));
+        }
+
+        [Fact]
+        public void TestSingleElement()
+        {
+            Int32Array array = new Int32Array.Builder().Append(1).Append(2).Build();
+            IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(new [] { array });
+            ArrowReaderVerifier.CompareArrays(array, actualArray);
+        }
+
+        private static IEnumerable<Tuple<List<IArrowArray>, IArrowArray>> GenerateTestData()
+        {
+            var targetTypes = new List<IArrowType>() {
+                    BooleanType.Default,
+                    Int8Type.Default,
+                    Int16Type.Default,
+                    Int32Type.Default,
+                    Int64Type.Default,
+                    UInt8Type.Default,
+                    UInt16Type.Default,
+                    UInt32Type.Default,
+                    UInt64Type.Default,
+                    FloatType.Default,
+                    DoubleType.Default,
+                    BinaryType.Default,
+                    StringType.Default,
+                    Date32Type.Default,
+                    Date64Type.Default,
+                    TimestampType.Default,
+                    new Decimal128Type(14, 10),
+                    new Decimal256Type(14,10),
+                    new ListType(Int64Type.Default),
+                    new StructType(new List<Field>{
+                        new Field.Builder().Name("Strings").DataType(StringType.Default).Nullable(true).Build(),
+                        new Field.Builder().Name("Ints").DataType(Int32Type.Default).Nullable(true).Build()
+                    }),
+                };
+
+            foreach (IArrowType type in targetTypes)
+            {
+                var creator = new TestDataGenerator();
+                type.Accept(creator);
+                yield return Tuple.Create(creator.TestTargetArrayList, creator.ExpectedArray);
+            }
+        }
+
+        private class TestDataGenerator :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<Int8Type>,
+            IArrowTypeVisitor<Int16Type>,
+            IArrowTypeVisitor<Int32Type>,
+            IArrowTypeVisitor<Int64Type>,
+            IArrowTypeVisitor<UInt8Type>,
+            IArrowTypeVisitor<UInt16Type>,
+            IArrowTypeVisitor<UInt32Type>,
+            IArrowTypeVisitor<UInt64Type>,
+            IArrowTypeVisitor<FloatType>,
+            IArrowTypeVisitor<DoubleType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<Decimal128Type>,
+            IArrowTypeVisitor<Decimal256Type>,
+            IArrowTypeVisitor<Date32Type>,
+            IArrowTypeVisitor<Date64Type>,
+            IArrowTypeVisitor<TimestampType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+
+            private List<List<int?>> _baseData;
+
+            private int _baseDataListCount;
+
+            private int _baseDataTotalElementCount;
+
+            public List<IArrowArray> TestTargetArrayList { get; }
+            public IArrowArray ExpectedArray { get; private set; }
+
+            public TestDataGenerator()
+            {
+                _baseData = new List<List<int?>> {
+                    new List<int?> { 1, 2, 3 },
+                    new List<int?> { 100, 101, null },
+                    new List<int?> { 11, null, 12 },
+                };
+
+                _baseDataListCount = _baseData.Count;
+                _baseDataTotalElementCount = _baseData.Sum(_ => _.Count);
+                TestTargetArrayList = new List<IArrowArray>(_baseDataListCount);
+            }
+
+            public void Visit(BooleanType type) => GenerateTestData<bool, BooleanArray, BooleanArray.Builder>(type, x => x % 2 == 0);
+            public void Visit(Int8Type type) => GenerateTestData<sbyte, Int8Array, Int8Array.Builder>(type, x => (sbyte)x);
+            public void Visit(Int16Type type) => GenerateTestData<short, Int16Array, Int16Array.Builder>(type, x => (short)x);
+            public void Visit(Int32Type type) => GenerateTestData<int, Int32Array, Int32Array.Builder>(type, x => x);
+            public void Visit(Int64Type type) => GenerateTestData<long, Int64Array, Int64Array.Builder>(type, x => x);
+            public void Visit(UInt8Type type) => GenerateTestData<byte, UInt8Array, UInt8Array.Builder>(type, x => (byte)x);
+            public void Visit(UInt16Type type) => GenerateTestData<ushort, UInt16Array, UInt16Array.Builder>(type, x => (ushort)x);
+            public void Visit(UInt32Type type) => GenerateTestData<uint, UInt32Array, UInt32Array.Builder>(type, x => (uint)x);
+            public void Visit(UInt64Type type) => GenerateTestData<ulong, UInt64Array, UInt64Array.Builder>(type, x => (ulong)x);
+            public void Visit(FloatType type) => GenerateTestData<float, FloatArray, FloatArray.Builder>(type, x => x);
+            public void Visit(DoubleType type) => GenerateTestData<double, DoubleArray, DoubleArray.Builder>(type, x => x);
+            public void Visit(Date32Type type) => GenerateTestData<DateTime, Date32Array, Date32Array.Builder>(type, x => DateTime.MinValue.AddDays(x));
+            public void Visit(Date64Type type) => GenerateTestData<DateTime, Date64Array, Date64Array.Builder>(type, x => DateTime.MinValue.AddDays(x));

Review comment:
       The `TestData` class is dependent on `Length` and unable to generate flexible data for now.
   For example, when a type is `Int32Type`, it can not create data started with a non-zero value like `[3,4,5]`.
   On the other hand, we need to create that kind of data for testing array concatenation.
   E.g. an expected concatenated array is `[0,1,2,3,4,5,6,7,null]` and base arrays are `[0,1,2]`, `[3,4,5]`, `[6.7.null]`.
   
   In order to reuse the `TestData` class, we need to modify it to be able to handle arbitrary data, but it might have a considerable impact.
   For the above reasons, I didn't reuse(modify) it.
   
   I think it is better to modify the `TestData` class and use it in `ArrowArrayConcatenatorTests` in future.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   @eerhardt 
   Thank you for the review!
   I have addressed your feedback.
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   The internal `ListArray.Builder` ctor was called instead of the public ctor in `ArrayBuilderTests.NestedListArrayBuilder()`. 
   Because of adding `InternalsVisibleTo` into `AssemblyInfo.cs`, `Arrow.Array.Test` could see the internal ctor and it had higher overload priority than the public ctor.
   
   ```
   public Builder(IArrowType valueDataType) ...
   internal Builder(ListType dataType) ...
   ```
   
   For the above reasons, I changed to use Reflection instead of internalsVisibleTo.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayConcatenatorTests.cs
##########
@@ -0,0 +1,383 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ArrowArrayConcatenatorTests
+    {
+        [Fact]
+        public void TestStandardCases()
+        {
+            foreach ((List<IArrowArray> testTargetArrayList, IArrowArray expectedArray) in GenerateTestData())
+            {
+                IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(testTargetArrayList);
+                ArrowReaderVerifier.CompareArrays(expectedArray, actualArray);
+            }
+        }
+
+        [Fact]
+        public void TestNullOrEmpty()
+        {
+            Assert.Null(ArrowArrayConcatenator.Concatenate(null));
+            Assert.Null(ArrowArrayConcatenator.Concatenate(new List<IArrowArray>()));
+        }
+
+        [Fact]
+        public void TestSingleElement()
+        {
+            Int32Array array = new Int32Array.Builder().Append(1).Append(2).Build();
+            IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(new [] { array });
+            ArrowReaderVerifier.CompareArrays(array, actualArray);
+        }
+
+        private static IEnumerable<Tuple<List<IArrowArray>, IArrowArray>> GenerateTestData()
+        {
+            var targetTypes = new List<IArrowType>() {
+                    BooleanType.Default,
+                    Int8Type.Default,
+                    Int16Type.Default,
+                    Int32Type.Default,
+                    Int64Type.Default,
+                    UInt8Type.Default,
+                    UInt16Type.Default,
+                    UInt32Type.Default,
+                    UInt64Type.Default,
+                    FloatType.Default,
+                    DoubleType.Default,
+                    BinaryType.Default,
+                    StringType.Default,
+                    Date32Type.Default,
+                    Date64Type.Default,
+                    TimestampType.Default,
+                    new Decimal128Type(14, 10),
+                    new Decimal256Type(14,10),
+                    new ListType(Int64Type.Default),
+                    new StructType(new List<Field>{
+                        new Field.Builder().Name("Strings").DataType(StringType.Default).Nullable(true).Build(),
+                        new Field.Builder().Name("Ints").DataType(Int32Type.Default).Nullable(true).Build()
+                    }),
+                };
+
+            foreach (IArrowType type in targetTypes)
+            {
+                var creator = new TestDataGenerator();
+                type.Accept(creator);
+                yield return Tuple.Create(creator.TestTargetArrayList, creator.ExpectedArray);
+            }
+        }
+
+        private class TestDataGenerator :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<Int8Type>,
+            IArrowTypeVisitor<Int16Type>,
+            IArrowTypeVisitor<Int32Type>,
+            IArrowTypeVisitor<Int64Type>,
+            IArrowTypeVisitor<UInt8Type>,
+            IArrowTypeVisitor<UInt16Type>,
+            IArrowTypeVisitor<UInt32Type>,
+            IArrowTypeVisitor<UInt64Type>,
+            IArrowTypeVisitor<FloatType>,
+            IArrowTypeVisitor<DoubleType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<Decimal128Type>,
+            IArrowTypeVisitor<Decimal256Type>,
+            IArrowTypeVisitor<Date32Type>,
+            IArrowTypeVisitor<Date64Type>,
+            IArrowTypeVisitor<TimestampType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+
+            private List<List<int?>> _baseData;
+
+            private int _baseDataListCount;
+
+            private int _baseDataTotalElementCount;
+
+            public List<IArrowArray> TestTargetArrayList { get; }
+            public IArrowArray ExpectedArray { get; private set; }
+
+            public TestDataGenerator()
+            {
+                _baseData = new List<List<int?>> {
+                    new List<int?> { 1, 2, 3 },
+                    new List<int?> { 100, 101, null },
+                    new List<int?> { 11, null, 12 },
+                };
+
+                _baseDataListCount = _baseData.Count;
+                _baseDataTotalElementCount = _baseData.Sum(_ => _.Count);
+                TestTargetArrayList = new List<IArrowArray>(_baseDataListCount);
+            }
+
+            public void Visit(BooleanType type) => GenerateTestData<bool, BooleanArray, BooleanArray.Builder>(type, x => x % 2 == 0);
+            public void Visit(Int8Type type) => GenerateTestData<sbyte, Int8Array, Int8Array.Builder>(type, x => (sbyte)x);
+            public void Visit(Int16Type type) => GenerateTestData<short, Int16Array, Int16Array.Builder>(type, x => (short)x);
+            public void Visit(Int32Type type) => GenerateTestData<int, Int32Array, Int32Array.Builder>(type, x => x);
+            public void Visit(Int64Type type) => GenerateTestData<long, Int64Array, Int64Array.Builder>(type, x => x);
+            public void Visit(UInt8Type type) => GenerateTestData<byte, UInt8Array, UInt8Array.Builder>(type, x => (byte)x);
+            public void Visit(UInt16Type type) => GenerateTestData<ushort, UInt16Array, UInt16Array.Builder>(type, x => (ushort)x);
+            public void Visit(UInt32Type type) => GenerateTestData<uint, UInt32Array, UInt32Array.Builder>(type, x => (uint)x);
+            public void Visit(UInt64Type type) => GenerateTestData<ulong, UInt64Array, UInt64Array.Builder>(type, x => (ulong)x);
+            public void Visit(FloatType type) => GenerateTestData<float, FloatArray, FloatArray.Builder>(type, x => x);
+            public void Visit(DoubleType type) => GenerateTestData<double, DoubleArray, DoubleArray.Builder>(type, x => x);
+            public void Visit(Date32Type type) => GenerateTestData<DateTime, Date32Array, Date32Array.Builder>(type, x => DateTime.MinValue.AddDays(x));
+            public void Visit(Date64Type type) => GenerateTestData<DateTime, Date64Array, Date64Array.Builder>(type, x => DateTime.MinValue.AddDays(x));

Review comment:
       The `TestData` class is unable to generate flexible data for now.
   For example, when a type is `Int32Type`, it can not generate data that start with a non-zero value like `[3,4,5]`.
   On the other hand, we need to create that kind of data for testing array concatenation.
   E.g. an expected concatenated array is `[0,1,2,3,4,5,6,7,null]` and base arrays are `[0,1,2]`, `[3,4,5]`, `[6,7,null]`. 
   
   In order to reuse the `TestData` class, we need to modify it to be able to handle arbitrary data, but it might have a considerable impact.
   For the above reasons, I didn't reuse(modify) it.
   
   I think it is better to modify the `TestData` class and use it in `ArrowArrayConcatenatorTests` in future.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    public static class ArrayDataConcatenator

Review comment:
       I see.
   I have changed them to `internal` and changed to use `InternalsVisibleTo`.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayConcatenatorTests.cs
##########
@@ -0,0 +1,383 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ArrowArrayConcatenatorTests
+    {
+        [Fact]
+        public void TestStandardCases()
+        {
+            foreach ((List<IArrowArray> testTargetArrayList, IArrowArray expectedArray) in GenerateTestData())
+            {
+                IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(testTargetArrayList);
+                ArrowReaderVerifier.CompareArrays(expectedArray, actualArray);
+            }
+        }
+
+        [Fact]
+        public void TestNullOrEmpty()
+        {
+            Assert.Null(ArrowArrayConcatenator.Concatenate(null));
+            Assert.Null(ArrowArrayConcatenator.Concatenate(new List<IArrowArray>()));
+        }
+
+        [Fact]
+        public void TestSingleElement()
+        {
+            Int32Array array = new Int32Array.Builder().Append(1).Append(2).Build();
+            IArrowArray actualArray = ArrowArrayConcatenator.Concatenate(new [] { array });
+            ArrowReaderVerifier.CompareArrays(array, actualArray);
+        }
+
+        private static IEnumerable<Tuple<List<IArrowArray>, IArrowArray>> GenerateTestData()
+        {
+            var targetTypes = new List<IArrowType>() {
+                    BooleanType.Default,
+                    Int8Type.Default,
+                    Int16Type.Default,
+                    Int32Type.Default,
+                    Int64Type.Default,
+                    UInt8Type.Default,
+                    UInt16Type.Default,
+                    UInt32Type.Default,
+                    UInt64Type.Default,
+                    FloatType.Default,
+                    DoubleType.Default,
+                    BinaryType.Default,
+                    StringType.Default,
+                    Date32Type.Default,
+                    Date64Type.Default,
+                    TimestampType.Default,
+                    new Decimal128Type(14, 10),
+                    new Decimal256Type(14,10),
+                    new ListType(Int64Type.Default),
+                    new StructType(new List<Field>{
+                        new Field.Builder().Name("Strings").DataType(StringType.Default).Nullable(true).Build(),
+                        new Field.Builder().Name("Ints").DataType(Int32Type.Default).Nullable(true).Build()
+                    }),
+                };
+
+            foreach (IArrowType type in targetTypes)
+            {
+                var creator = new TestDataGenerator();
+                type.Accept(creator);
+                yield return Tuple.Create(creator.TestTargetArrayList, creator.ExpectedArray);
+            }
+        }
+
+        private class TestDataGenerator :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<Int8Type>,
+            IArrowTypeVisitor<Int16Type>,
+            IArrowTypeVisitor<Int32Type>,
+            IArrowTypeVisitor<Int64Type>,
+            IArrowTypeVisitor<UInt8Type>,
+            IArrowTypeVisitor<UInt16Type>,
+            IArrowTypeVisitor<UInt32Type>,
+            IArrowTypeVisitor<UInt64Type>,
+            IArrowTypeVisitor<FloatType>,
+            IArrowTypeVisitor<DoubleType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<Decimal128Type>,
+            IArrowTypeVisitor<Decimal256Type>,
+            IArrowTypeVisitor<Date32Type>,
+            IArrowTypeVisitor<Date64Type>,
+            IArrowTypeVisitor<TimestampType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+
+            private List<List<int?>> _baseData;
+
+            private int _baseDataListCount;
+
+            private int _baseDataTotalElementCount;
+
+            public List<IArrowArray> TestTargetArrayList { get; }
+            public IArrowArray ExpectedArray { get; private set; }
+
+            public TestDataGenerator()
+            {
+                _baseData = new List<List<int?>> {
+                    new List<int?> { 1, 2, 3 },
+                    new List<int?> { 100, 101, null },
+                    new List<int?> { 11, null, 12 },
+                };
+
+                _baseDataListCount = _baseData.Count;
+                _baseDataTotalElementCount = _baseData.Sum(_ => _.Count);
+                TestTargetArrayList = new List<IArrowArray>(_baseDataListCount);
+            }
+
+            public void Visit(BooleanType type) => GenerateTestData<bool, BooleanArray, BooleanArray.Builder>(type, x => x % 2 == 0);
+            public void Visit(Int8Type type) => GenerateTestData<sbyte, Int8Array, Int8Array.Builder>(type, x => (sbyte)x);
+            public void Visit(Int16Type type) => GenerateTestData<short, Int16Array, Int16Array.Builder>(type, x => (short)x);
+            public void Visit(Int32Type type) => GenerateTestData<int, Int32Array, Int32Array.Builder>(type, x => x);
+            public void Visit(Int64Type type) => GenerateTestData<long, Int64Array, Int64Array.Builder>(type, x => x);
+            public void Visit(UInt8Type type) => GenerateTestData<byte, UInt8Array, UInt8Array.Builder>(type, x => (byte)x);
+            public void Visit(UInt16Type type) => GenerateTestData<ushort, UInt16Array, UInt16Array.Builder>(type, x => (ushort)x);
+            public void Visit(UInt32Type type) => GenerateTestData<uint, UInt32Array, UInt32Array.Builder>(type, x => (uint)x);
+            public void Visit(UInt64Type type) => GenerateTestData<ulong, UInt64Array, UInt64Array.Builder>(type, x => (ulong)x);
+            public void Visit(FloatType type) => GenerateTestData<float, FloatArray, FloatArray.Builder>(type, x => x);
+            public void Visit(DoubleType type) => GenerateTestData<double, DoubleArray, DoubleArray.Builder>(type, x => x);
+            public void Visit(Date32Type type) => GenerateTestData<DateTime, Date32Array, Date32Array.Builder>(type, x => DateTime.MinValue.AddDays(x));
+            public void Visit(Date64Type type) => GenerateTestData<DateTime, Date64Array, Date64Array.Builder>(type, x => DateTime.MinValue.AddDays(x));

Review comment:
       The `TestData` class is dependent on `Length` and unable to generate flexible data for now.
   For example, when a type is `Int32Type`, it can not create data started with a non-zero value like `[2,3,4]`.
   On the other hand, we need to create that kind of data for testing array concatenation.
   E.g. `[0,1,2]`, `[3,4,5]`, `[6.7.null]` and `[0,1,2,3,4,5,6,7,null]`.
   
   In order to reuse the `TestData` class, we need to modify it to be able to handle arbitrary data, but it might have a considerable impact.
   For the above reasons, I didn't reuse(modify) the `TestData` class.
   
   I think it is better to modify the `TestData` class and use it in `ArrowArrayConcatenatorTests` in future.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] eerhardt commented on a change in pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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



##########
File path: csharp/test/Apache.Arrow.Tests/ArrayDataConcatenatorTests.cs
##########
@@ -0,0 +1,52 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Collections.Generic;
+using System.Reflection;
+using Apache.Arrow.Memory;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ArrayDataConcatenatorTests
+    {
+        [Fact]
+        public void TestNullOrEmpty()
+        {
+            Assert.Null(ArrayDataConcatenatorReflector.InvokeConcatenate(null));
+            Assert.Null(ArrayDataConcatenatorReflector.InvokeConcatenate(new List<ArrayData>()));
+        }
+
+        [Fact]
+        public void TestSingleElement()
+        {
+            Int32Array array = new Int32Array.Builder().Append(1).Append(2).Build();
+            ArrayData actualArray = ArrayDataConcatenatorReflector.InvokeConcatenate(new[] { array.Data });
+            ArrowReaderVerifier.CompareArrays(array, ArrowArrayFactory.BuildArray(actualArray));
+        }
+
+        private static class ArrayDataConcatenatorReflector
+        {
+            private static readonly Type s_arrayDataConcatenatorType = Assembly.Load("Apache.Arrow").GetType("Apache.Arrow.ArrayDataConcatenator");

Review comment:
       ```suggestion
               private static readonly Type s_arrayDataConcatenatorType = typeof(ArrayData).Assembly.GetType("Apache.Arrow.ArrayDataConcatenator");
   ```

##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayBuilderFactoryReflector.cs
##########
@@ -0,0 +1,32 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Reflection;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow.Tests
+{
+    static class ArrayArrayBuilderFactoryReflector
+    {
+        private static readonly Type s_arrowArrayBuilderFactoryType = Assembly.Load("Apache.Arrow").GetType("Apache.Arrow.ArrowArrayBuilderFactory");

Review comment:
       Do we need to hold onto this type statically in a field? It could just be inlined into the MethodInfo call below and the field could be removed.

##########
File path: csharp/src/Apache.Arrow/Arrays/ListArray.cs
##########
@@ -77,7 +77,7 @@ public Builder AppendNull()
 
             public ListArray Build(MemoryAllocator allocator = default)
             {
-                Append();
+                ValueOffsetsBufferBuilder.Append(ValueBuilder.Length);

Review comment:
       Can you write a unit test for this to ensure it stays fixed?

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -144,7 +154,7 @@ private void ReadDictionaryBatch(Flatbuf.DictionaryBatch dictionaryBatch, ByteBu
 
             if (dictionaryBatch.IsDelta)
             {
-                throw new NotImplementedException("Dictionary delta is not supported yet");
+                DictionaryMemo.AddDeltaDictionary(id, arrays[0], _allocator);

Review comment:
       Note, once my integration change #10973 is merged, you can enable the dictionary integration tests to make sure dictionaries are working across C# and the other languages.

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    static class ArrayDataConcatenator
+    {
+        internal static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatenationVisitor = new ArrayDataConcatenationVisitor(arrayDataList, allocator);
+
+            IArrowType type = arrayDataList[0].DataType;
+            type.Accept(arrowArrayConcatenationVisitor);
+
+            return arrowArrayConcatenationVisitor.Result;
+        }
+
+        private class ArrayDataConcatenationVisitor :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<FixedWidthType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+            public ArrayData Result { get; private set; }
+            private readonly IReadOnlyList<ArrayData> _arrayDataList;
+            private readonly int _totalLength;
+            private readonly int _totalNullCount;
+            private readonly MemoryAllocator _allocator;
+
+            public ArrayDataConcatenationVisitor(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+            {
+                _arrayDataList = arrayDataList;
+                _allocator = allocator;
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    _totalLength += arrayData.Length;
+                    _totalNullCount += arrayData.NullCount;
+                }
+            }
+
+            public void Visit(BooleanType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateBitmapBuffer(1);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(FixedWidthType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateFixedWidthTypeValueBuffer(type);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(BinaryType type) => ConcatenateVariableBinaryArrayData(type);
+
+            public void Visit(StringType type) => ConcatenateVariableBinaryArrayData(type);
+
+            public void Visit(ListType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrayData child = Concatenate(SelectChildren(0), _allocator);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer }, new[] { child });
+            }
+
+            public void Visit(StructType type)
+            {
+                CheckData(type, 1);
+                List<ArrayData> children = new List<ArrayData>(type.Fields.Count);
+
+                for (int i = 0; i < type.Fields.Count; i++)
+                {
+                    children.Add(Concatenate(SelectChildren(i), _allocator));
+                }
+
+                Result = new ArrayData(type, _arrayDataList[0].Length, _arrayDataList[0].NullCount, 0, _arrayDataList[0].Buffers, children);
+            }
+
+            public void Visit(IArrowType type)
+            {
+                throw new NotImplementedException($"Concatination for {type.Name} is not supported yet.");
+            }
+
+            private void CheckData(IArrowType type, int expectedBufferCount)
+            {
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    arrayData.EnsureDataType(type.TypeId);
+                    arrayData.EnsureBufferCount(expectedBufferCount);
+                }
+            }
+
+            private void ConcatenateVariableBinaryArrayData(IArrowType type)
+            {
+                CheckData(type, 3);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrowBuffer valueBuffer = ConcatenateVariableBinaryValueBuffer();
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer, valueBuffer });
+            }
+
+            private ArrowBuffer ConcatenateValidityBuffer()
+            {
+                if (_totalNullCount == 0)
+                {
+                    return ArrowBuffer.Empty;
+                }
+
+                return ConcatenateBitmapBuffer(0);
+            }
+
+            private ArrowBuffer ConcatenateBitmapBuffer(int bufferIndex)
+            {
+                var builder = new ArrowBuffer.BitmapBuilder(_totalLength);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int length = arrayData.Length;
+                    ReadOnlySpan<byte> span = arrayData.Buffers[bufferIndex].Span;
+
+                    for (int i = 0; i < length; i++)
+                    {
+                        builder.Append(span.IsEmpty || BitUtility.GetBit(span, i));
+                    }
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcatenateFixedWidthTypeValueBuffer(FixedWidthType type)
+            {
+                int typeByteWidth = type.BitWidth / 8;
+                var builder = new ArrowBuffer.Builder<byte>(_totalLength * typeByteWidth);
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int length = arrayData.Length;
+                    int byteLength = length * typeByteWidth;
+
+                    builder.Append(arrayData.Buffers[1].Span.Slice(0, byteLength));
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcatenateVariableBinaryValueBuffer()
+            {
+                var builder = new ArrowBuffer.Builder<byte>();
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    int lastOffset = arrayData.Buffers[1].Span.CastTo<int>()[arrayData.Length];
+                    builder.Append(arrayData.Buffers[2].Span.Slice(0, lastOffset));
+                }
+
+                return builder.Build(_allocator);
+            }
+
+            private ArrowBuffer ConcateneteOffsetBuffer()

Review comment:
       ```suggestion
               private ArrowBuffer ConcatenateOffsetBuffer()
   ```

##########
File path: csharp/test/Apache.Arrow.Tests/ArrowArrayBuilderFactoryReflector.cs
##########
@@ -0,0 +1,32 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+using System;
+using System.Reflection;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow.Tests
+{
+    static class ArrayArrayBuilderFactoryReflector
+    {
+        private static readonly Type s_arrowArrayBuilderFactoryType = Assembly.Load("Apache.Arrow").GetType("Apache.Arrow.ArrowArrayBuilderFactory");

Review comment:
       ```suggestion
           private static readonly Type s_arrowArrayBuilderFactoryType = typeof(ArrayData).Assembly.GetType("Apache.Arrow.ArrowArrayBuilderFactory");
   ```

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayDataConcatenator.cs
##########
@@ -0,0 +1,240 @@
+// 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.Memory;
+using Apache.Arrow.Types;
+using System;
+using System.Collections.Generic;
+
+namespace Apache.Arrow
+{
+    static class ArrayDataConcatenator
+    {
+        internal static ArrayData Concatenate(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+        {
+            if (arrayDataList == null || arrayDataList.Count == 0)
+            {
+                return null;
+            }
+
+            if (arrayDataList.Count == 1)
+            {
+                return arrayDataList[0];
+            }
+
+            var arrowArrayConcatenationVisitor = new ArrayDataConcatenationVisitor(arrayDataList, allocator);
+
+            IArrowType type = arrayDataList[0].DataType;
+            type.Accept(arrowArrayConcatenationVisitor);
+
+            return arrowArrayConcatenationVisitor.Result;
+        }
+
+        private class ArrayDataConcatenationVisitor :
+            IArrowTypeVisitor<BooleanType>,
+            IArrowTypeVisitor<FixedWidthType>,
+            IArrowTypeVisitor<BinaryType>,
+            IArrowTypeVisitor<StringType>,
+            IArrowTypeVisitor<ListType>,
+            IArrowTypeVisitor<StructType>
+        {
+            public ArrayData Result { get; private set; }
+            private readonly IReadOnlyList<ArrayData> _arrayDataList;
+            private readonly int _totalLength;
+            private readonly int _totalNullCount;
+            private readonly MemoryAllocator _allocator;
+
+            public ArrayDataConcatenationVisitor(IReadOnlyList<ArrayData> arrayDataList, MemoryAllocator allocator = default)
+            {
+                _arrayDataList = arrayDataList;
+                _allocator = allocator;
+
+                foreach (ArrayData arrayData in _arrayDataList)
+                {
+                    _totalLength += arrayData.Length;
+                    _totalNullCount += arrayData.NullCount;
+                }
+            }
+
+            public void Visit(BooleanType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateBitmapBuffer(1);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(FixedWidthType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer valueBuffer = ConcatenateFixedWidthTypeValueBuffer(type);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, valueBuffer });
+            }
+
+            public void Visit(BinaryType type) => ConcatenateVariableBinaryArrayData(type);
+
+            public void Visit(StringType type) => ConcatenateVariableBinaryArrayData(type);
+
+            public void Visit(ListType type)
+            {
+                CheckData(type, 2);
+                ArrowBuffer validityBuffer = ConcatenateValidityBuffer();
+                ArrowBuffer offsetBuffer = ConcateneteOffsetBuffer();
+                ArrayData child = Concatenate(SelectChildren(0), _allocator);
+
+                Result = new ArrayData(type, _totalLength, _totalNullCount, 0, new ArrowBuffer[] { validityBuffer, offsetBuffer }, new[] { child });
+            }
+
+            public void Visit(StructType type)
+            {
+                CheckData(type, 1);
+                List<ArrayData> children = new List<ArrayData>(type.Fields.Count);
+
+                for (int i = 0; i < type.Fields.Count; i++)
+                {
+                    children.Add(Concatenate(SelectChildren(i), _allocator));
+                }
+
+                Result = new ArrayData(type, _arrayDataList[0].Length, _arrayDataList[0].NullCount, 0, _arrayDataList[0].Buffers, children);
+            }
+
+            public void Visit(IArrowType type)
+            {
+                throw new NotImplementedException($"Concatination for {type.Name} is not supported yet.");

Review comment:
       ```suggestion
                   throw new NotImplementedException($"Concatenation for {type.Name} is not supported yet.");
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   @eerhardt 
   Sorry, I pushed the 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] HashidaTKS commented on pull request #10990: ARROW-13704: [C#] Add support for reading streaming format delta dictionaries

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


   The internal `ListArray.Builder` ctor was called instead of the public ctor in `ArrayBuilderTests.NestedListArrayBuilder()`. 
   Because of adding `InternalsVisibleTo` into `AssemblyInfo.cs`, `Arrow.Array.Test` could see the internal ctor and it had higher overload priority than the public ctor.
   
   ```
   public Builder(IArrowType valueDataType) ...
   internal Builder(ListType dataType) ...
   ```
   
   For the above reasons, I changed to use Reflection instead of internalsVisibleTo.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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