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/06/18 21:17:36 UTC

[GitHub] [arrow] HashidaTKS commented on a change in pull request #10527: ARROW-6870: [C#] Add Support for Dictionary Arrays and Dictionary Encoding

HashidaTKS commented on a change in pull request #10527:
URL: https://github.com/apache/arrow/pull/10527#discussion_r653778602



##########
File path: csharp/src/Apache.Arrow/RecordBatch.cs
##########
@@ -29,7 +29,7 @@ public partial class RecordBatch : IDisposable
         public int Length { get; }
 
         private readonly IMemoryOwner<byte> _memoryOwner;
-        private readonly IList<IArrowArray> _arrays;
+        internal readonly IReadOnlyList<IArrowArray> _arrays;

Review comment:
       Fixed it.
   
   There was already a public property named `Arrays` so I named the internal property `_Arrays`.

##########
File path: csharp/src/Apache.Arrow/Arrays/DictionaryArray.cs
##########
@@ -0,0 +1,67 @@
+// 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.IO;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DictionaryArray : Array
+    {
+        public IArrowArray Dictionary { get; }
+        public IArrowArray Indices { get; }
+        public ArrowBuffer IndicesBuffer => Data.Buffers[1];
+
+        public DictionaryArray(IArrowType dataType, int length,
+            ArrowBuffer valueOffsetsBuffer, IArrowArray value,
+            ArrowBuffer nullBitmapBuffer, int nullCount = 0, int offset = 0)
+            : this(new ArrayData(dataType, length, nullCount, offset,
+                new[] { nullBitmapBuffer, valueOffsetsBuffer }, new[] { value.Data }, value.Data.Dictionary))
+        {
+        }
+
+        public DictionaryArray(ArrayData data) : base(data)
+        {
+            data.EnsureBufferCount(2);
+            data.EnsureDataType(ArrowTypeId.Dictionary);
+
+            var dicType = data.DataType as DictionaryType;
+            data.Dictionary.EnsureDataType(dicType.ValueType.TypeId);
+
+            ArrayData indicesData = new ArrayData(dicType.IndexType, data.Length, data.NullCount, data.Offset, data.Buffers, data.Children);
+
+            Indices = ArrowArrayFactory.BuildArray(indicesData);
+            Dictionary = ArrowArrayFactory.BuildArray(data.Dictionary);
+        }
+
+        public DictionaryArray(IArrowType dataType, IArrowArray indicesArray, IArrowArray dictionary, bool ordered = false) :

Review comment:
       Fixed it.

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -29,6 +29,13 @@ internal abstract class ArrowReaderImplementation : IDisposable
     {
         public Schema Schema { get; protected set; }
         protected bool HasReadSchema => Schema != null;
+        protected bool HasReadInitialDictionary { get; set; }

Review comment:
       Fixed it.

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -29,6 +29,13 @@ internal abstract class ArrowReaderImplementation : IDisposable
     {
         public Schema Schema { get; protected set; }
         protected bool HasReadSchema => Schema != null;
+        protected bool HasReadInitialDictionary { get; set; }
+        protected readonly DictionaryMemo _dictionaryMemo;
+
+        public ArrowReaderImplementation()
+        {
+            _dictionaryMemo = new DictionaryMemo();

Review comment:
       I have created the `LazyCreator` class and changed `DictionaryMemo _dictionaryMemo` to `LazyCreator<DictionaryMemo> _lazyDictionaryMemo`.
   However, I am wondering whether this is the best way to lazy load because it need allocate a `LazyCreator` object.
   Is there any other good idea?
   

##########
File path: csharp/src/Apache.Arrow/Arrays/DictionaryArray.cs
##########
@@ -0,0 +1,67 @@
+// 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.IO;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DictionaryArray : Array
+    {
+        public IArrowArray Dictionary { get; }
+        public IArrowArray Indices { get; }
+        public ArrowBuffer IndicesBuffer => Data.Buffers[1];
+
+        public DictionaryArray(IArrowType dataType, int length,
+            ArrowBuffer valueOffsetsBuffer, IArrowArray value,
+            ArrowBuffer nullBitmapBuffer, int nullCount = 0, int offset = 0)
+            : this(new ArrayData(dataType, length, nullCount, offset,
+                new[] { nullBitmapBuffer, valueOffsetsBuffer }, new[] { value.Data }, value.Data.Dictionary))
+        {
+        }
+
+        public DictionaryArray(ArrayData data) : base(data)
+        {
+            data.EnsureBufferCount(2);
+            data.EnsureDataType(ArrowTypeId.Dictionary);
+
+            var dicType = data.DataType as DictionaryType;

Review comment:
       Fixed it.

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayData.cs
##########
@@ -30,31 +30,34 @@ public sealed class ArrayData : IDisposable
         public readonly int Offset;
         public readonly ArrowBuffer[] Buffers;
         public readonly ArrayData[] Children;
+        public readonly ArrayData Dictionary; //Only used for dictionary type
 
         public ArrayData(
             IArrowType dataType,
             int length, int nullCount = 0, int offset = 0,
-            IEnumerable<ArrowBuffer> buffers = null, IEnumerable<ArrayData> children = null)
+            IEnumerable<ArrowBuffer> buffers = null, IEnumerable<ArrayData> children = null, ArrayData dictionary = null)

Review comment:
       Fixed it.

##########
File path: csharp/src/Apache.Arrow/Arrays/DictionaryArray.cs
##########
@@ -0,0 +1,67 @@
+// 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.IO;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DictionaryArray : Array
+    {
+        public IArrowArray Dictionary { get; }
+        public IArrowArray Indices { get; }
+        public ArrowBuffer IndicesBuffer => Data.Buffers[1];
+
+        public DictionaryArray(IArrowType dataType, int length,
+            ArrowBuffer valueOffsetsBuffer, IArrowArray value,
+            ArrowBuffer nullBitmapBuffer, int nullCount = 0, int offset = 0)
+            : this(new ArrayData(dataType, length, nullCount, offset,
+                new[] { nullBitmapBuffer, valueOffsetsBuffer }, new[] { value.Data }, value.Data.Dictionary))
+        {
+        }
+
+        public DictionaryArray(ArrayData data) : base(data)
+        {
+            data.EnsureBufferCount(2);
+            data.EnsureDataType(ArrowTypeId.Dictionary);
+
+            var dicType = data.DataType as DictionaryType;
+            data.Dictionary.EnsureDataType(dicType.ValueType.TypeId);

Review comment:
       added a null check.

##########
File path: csharp/src/Apache.Arrow/Arrays/DictionaryArray.cs
##########
@@ -0,0 +1,67 @@
+// 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.IO;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DictionaryArray : Array
+    {
+        public IArrowArray Dictionary { get; }
+        public IArrowArray Indices { get; }
+        public ArrowBuffer IndicesBuffer => Data.Buffers[1];
+
+        public DictionaryArray(IArrowType dataType, int length,

Review comment:
       removed it.

##########
File path: csharp/src/Apache.Arrow/Arrays/DictionaryArray.cs
##########
@@ -0,0 +1,67 @@
+// 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.IO;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DictionaryArray : Array
+    {
+        public IArrowArray Dictionary { get; }
+        public IArrowArray Indices { get; }
+        public ArrowBuffer IndicesBuffer => Data.Buffers[1];
+
+        public DictionaryArray(IArrowType dataType, int length,

Review comment:
       Removed it.

##########
File path: csharp/src/Apache.Arrow/Arrays/DictionaryArray.cs
##########
@@ -0,0 +1,67 @@
+// 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.IO;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    public class DictionaryArray : Array
+    {
+        public IArrowArray Dictionary { get; }
+        public IArrowArray Indices { get; }
+        public ArrowBuffer IndicesBuffer => Data.Buffers[1];
+
+        public DictionaryArray(IArrowType dataType, int length,
+            ArrowBuffer valueOffsetsBuffer, IArrowArray value,
+            ArrowBuffer nullBitmapBuffer, int nullCount = 0, int offset = 0)
+            : this(new ArrayData(dataType, length, nullCount, offset,
+                new[] { nullBitmapBuffer, valueOffsetsBuffer }, new[] { value.Data }, value.Data.Dictionary))
+        {
+        }
+
+        public DictionaryArray(ArrayData data) : base(data)
+        {
+            data.EnsureBufferCount(2);
+            data.EnsureDataType(ArrowTypeId.Dictionary);
+
+            var dicType = data.DataType as DictionaryType;
+            data.Dictionary.EnsureDataType(dicType.ValueType.TypeId);

Review comment:
       Added a null check.

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowStreamWriter.cs
##########
@@ -216,10 +231,10 @@ private void CreateSelfAndChildrenFieldNodes(ArrayData data)
             Flatbuf.FieldNode.CreateFieldNode(Builder, data.Length, data.NullCount);
         }
 
-        private int CountAllNodes()
+        private int CountAllNodes(IReadOnlyDictionary<string, Field> fields)

Review comment:
       Fixed it.

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowStreamWriter.cs
##########
@@ -399,6 +439,95 @@ private protected void WriteRecordBatchInternal(RecordBatch recordBatch)
             return Tuple.Create(recordBatchBuilder, fieldNodesVectorOffset);
         }
 
+
+        private protected void WriteDictionaries(RecordBatch recordBatch)
+        {
+            foreach (Field field in recordBatch.Schema.Fields.Values)
+            {
+                WriteDictionary(field);
+            }
+        }
+
+        private protected void WriteDictionary(Field field)
+        {
+            if (field.DataType.TypeId != ArrowTypeId.Dictionary)
+            {
+                if (field.DataType is NestedType nestedType)
+                {
+                    foreach (Field child in nestedType.Fields)
+                    {
+                        WriteDictionary(child);
+                    }
+                }
+                return;
+            }
+
+            (ArrowRecordBatchFlatBufferBuilder recordBatchBuilder, Offset<Flatbuf.DictionaryBatch> dictionaryBatchOffset) =
+                CreateDictionaryBatchOffset(field);
+
+            WriteMessage(Flatbuf.MessageHeader.DictionaryBatch,
+                dictionaryBatchOffset, recordBatchBuilder.TotalLength);
+
+            WriteBufferData(recordBatchBuilder.Buffers);
+        }
+
+        private protected async Task WriteDictionariesAsync(RecordBatch recordBatch, CancellationToken cancellationToken = default)
+        {
+            foreach (Field field in recordBatch.Schema.Fields.Values)
+            {
+                await WriteDictionaryAsync(field, cancellationToken).ConfigureAwait(false);
+            }
+        }
+
+        private protected async Task WriteDictionaryAsync(Field field, CancellationToken cancellationToken = default)
+        {
+            if (field.DataType.TypeId != ArrowTypeId.Dictionary)
+            {
+                if (field.DataType is NestedType nestedType)
+                {
+                    foreach (Field child in nestedType.Fields)
+                    {
+                        await WriteDictionaryAsync(child, cancellationToken).ConfigureAwait(false);
+                    }
+                }
+                return;
+            }
+
+            (ArrowRecordBatchFlatBufferBuilder recordBatchBuilder, Offset<Flatbuf.DictionaryBatch> dictionaryBatchOffset) =
+                CreateDictionaryBatchOffset(field);
+
+            await WriteMessageAsync(Flatbuf.MessageHeader.DictionaryBatch,
+                dictionaryBatchOffset, recordBatchBuilder.TotalLength, cancellationToken).ConfigureAwait(false);
+
+            await WriteBufferDataAsync(recordBatchBuilder.Buffers, cancellationToken).ConfigureAwait(false);
+        }
+
+        private protected Tuple<ArrowRecordBatchFlatBufferBuilder, Offset<Flatbuf.DictionaryBatch>> CreateDictionaryBatchOffset(Field field)

Review comment:
       Fixed it.

##########
File path: csharp/src/Apache.Arrow/Ipc/DictionaryMemo.cs
##########
@@ -0,0 +1,113 @@
+// 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 Apache.Arrow.Types;
+
+namespace Apache.Arrow.Ipc
+{
+    class DictionaryMemo
+    {
+        private Dictionary<long, IArrowArray> _idToDictionary;
+        private Dictionary<long, IArrowType> _idToValueType;
+        private Dictionary<Field, long> _fieldToId;
+
+        public DictionaryMemo()
+        {
+            _idToDictionary = new Dictionary<long, IArrowArray>();
+            _idToValueType = new Dictionary<long, IArrowType>();
+            _fieldToId = new Dictionary<Field, long>();
+        }
+
+        public IArrowType GetDictionaryType(long id)
+        {
+            if (!_idToValueType.TryGetValue(id, out IArrowType type))
+            {
+                throw new ArgumentException($"Dictionary with id {id} not found");
+            }
+            return type;
+        }
+
+        public IArrowArray GetDictionary(long id)
+        {
+            if (!_idToDictionary.TryGetValue(id, out IArrowArray dictionary))
+            {
+                throw new ArgumentException($"Dictionary with id {id} not found");
+            }
+            return dictionary;
+        }
+
+        public void AddField(long id, Field field)
+        {
+            if (_fieldToId.ContainsKey(field))
+            {
+                throw new ArgumentException($"Field {field.Name} is already in Memo");
+            }
+
+            if (field.DataType.TypeId != ArrowTypeId.Dictionary)
+            {
+                throw new ArgumentException($"Field type is not DictionaryType: Name={field.Name}, {field.DataType.Name}");
+            }
+
+            IArrowType valueType = ((DictionaryType)field.DataType).ValueType;
+
+            if (_idToValueType.TryGetValue(id, out IArrowType valueTypeInDic))
+            {
+                if (valueType != valueTypeInDic)
+                {
+                    throw new ArgumentException($"Field type {field.DataType.Name} does not match the existing type {valueTypeInDic})");
+                }
+            }
+
+            _fieldToId.Add(field, id);
+            _idToValueType.Add(id, valueType);
+        }
+
+        public long GetId(Field field)
+        {
+            if (!_fieldToId.TryGetValue(field, out long id))
+            {
+                throw new ArgumentException($"Field with name {field.Name} not found");
+            }
+            return id;
+        }
+
+        public long GetOrAssignId(Field field)
+        {
+            if (!_fieldToId.TryGetValue(field, out long id))
+            {
+                id = _fieldToId.Count + 1;
+                AddField(id, field);
+            }
+            return id;
+        }
+
+        public void AddOrReplaceDictionary(long id, IArrowArray dictionary)
+        {
+            _idToDictionary[id] = dictionary;
+        }
+
+        public void AddDictionaryDelta(long id, IArrowArray dictionary)

Review comment:
       Fixed it.

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayData.cs
##########
@@ -30,10 +30,25 @@ public sealed class ArrayData : IDisposable
         public readonly int Offset;
         public readonly ArrowBuffer[] Buffers;
         public readonly ArrayData[] Children;
+        public readonly ArrayData Dictionary; //Only used for dictionary type
 
         public ArrayData(
             IArrowType dataType,
             int length, int nullCount = 0, int offset = 0,
+            IEnumerable<ArrowBuffer> buffers = null, IEnumerable<ArrayData> children = null) :
+            this(dataType, null, length, nullCount, offset, buffers, children)
+        { }
+
+        public ArrayData(
+            IArrowType dataType,
+            int length, int nullCount = 0, int offset = 0,
+            ArrowBuffer[] buffers = null, ArrayData[] children = null) :
+            this(dataType, null, length, nullCount, offset, buffers, children)
+        { }
+
+        public ArrayData(
+            IArrowType dataType, ArrayData dictionary,

Review comment:
       To avoid the ambiguity between ctors, `dictionary` is positioned the second argument...

##########
File path: csharp/test/Apache.Arrow.Tests/ArrowStreamReaderTests.cs
##########
@@ -68,7 +68,7 @@ public async Task Ctor_MemoryPool_AllocatesFromPool(bool shouldLeaveOpen)
                 ArrowStreamReader reader = new ArrowStreamReader(stream, memoryPool, shouldLeaveOpen);
                 reader.ReadNextRecordBatch();
 
-                Assert.Equal(1, memoryPool.Statistics.Allocations);
+                Assert.Equal(2, memoryPool.Statistics.Allocations);

Review comment:
       Memo:
   `_allocator` is called when reading dictionary batches and record batches.
   The expected value of `memoryPool.Statistics.Allocations` is `2` at this context.

##########
File path: csharp/src/Apache.Arrow/Ipc/ArrowReaderImplementation.cs
##########
@@ -29,6 +29,13 @@ internal abstract class ArrowReaderImplementation : IDisposable
     {
         public Schema Schema { get; protected set; }
         protected bool HasReadSchema => Schema != null;
+        protected bool HasReadInitialDictionary { get; set; }
+        protected readonly DictionaryMemo _dictionaryMemo;
+
+        public ArrowReaderImplementation()
+        {
+            _dictionaryMemo = new DictionaryMemo();

Review comment:
       I have created a `LazyCreator` class and changed `DictionaryMemo _dictionaryMemo` to `LazyCreator<DictionaryMemo> _lazyDictionaryMemo`.
   However, I am wondering whether this is the best way to lazy load because it need allocate a `LazyCreator` object.
   Is there any other good idea?
   

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayData.cs
##########
@@ -30,10 +30,25 @@ public sealed class ArrayData : IDisposable
         public readonly int Offset;
         public readonly ArrowBuffer[] Buffers;
         public readonly ArrayData[] Children;
+        public readonly ArrayData Dictionary; //Only used for dictionary type
 
         public ArrayData(
             IArrowType dataType,
             int length, int nullCount = 0, int offset = 0,
+            IEnumerable<ArrowBuffer> buffers = null, IEnumerable<ArrayData> children = null) :
+            this(dataType, null, length, nullCount, offset, buffers, children)
+        { }
+
+        public ArrayData(
+            IArrowType dataType,
+            int length, int nullCount = 0, int offset = 0,
+            ArrowBuffer[] buffers = null, ArrayData[] children = null) :
+            this(dataType, null, length, nullCount, offset, buffers, children)
+        { }
+
+        public ArrayData(
+            IArrowType dataType, ArrayData dictionary,

Review comment:
       To avoid the ambiguity between ctors, `dictionary` is positioned to the second argument...

##########
File path: csharp/src/Apache.Arrow/Arrays/ArrayData.cs
##########
@@ -30,10 +30,25 @@ public sealed class ArrayData : IDisposable
         public readonly int Offset;
         public readonly ArrowBuffer[] Buffers;
         public readonly ArrayData[] Children;
+        public readonly ArrayData Dictionary; //Only used for dictionary type
 
         public ArrayData(
             IArrowType dataType,
             int length, int nullCount = 0, int offset = 0,
+            IEnumerable<ArrowBuffer> buffers = null, IEnumerable<ArrayData> children = null) :
+            this(dataType, null, length, nullCount, offset, buffers, children)
+        { }
+
+        public ArrayData(
+            IArrowType dataType,
+            int length, int nullCount = 0, int offset = 0,
+            ArrowBuffer[] buffers = null, ArrayData[] children = null) :
+            this(dataType, null, length, nullCount, offset, buffers, children)
+        { }
+
+        public ArrayData(
+            IArrowType dataType, ArrayData dictionary,

Review comment:
       To avoid the ambiguity between ctors, `dictionary` is positioned to the second argument.
   I don't think it's a very good position in term of the meaning...




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

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