You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2019/06/18 18:04:58 UTC

[arrow] branch master updated: ARROW-4719: [C#] Implement ChunkedArray, Column and Table in C#

This is an automated email from the ASF dual-hosted git repository.

wesm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 54e04d0  ARROW-4719: [C#] Implement ChunkedArray, Column and Table in C#
54e04d0 is described below

commit 54e04d0f7feca22a133dc7450e64b8b853d0c15e
Author: Prashanth Govindarajan <pr...@microsoft.com>
AuthorDate: Tue Jun 18 13:04:51 2019 -0500

    ARROW-4719: [C#] Implement ChunkedArray, Column and Table in C#
    
    Port ChunkedArray, Column and Table to C#
    
    @eerhardt @chutchinson
    
    Author: Prashanth Govindarajan <pr...@microsoft.com>
    
    Closes #3806 from pgovind/Table and squashes the following commits:
    
    8aeec792d <Prashanth Govindarajan> sq
    541da746e <Prashanth Govindarajan> sq
    c16ae9dcd <Prashanth Govindarajan> Data access for Column
    391b7b073 <Prashanth Govindarajan> sq
    c00f727ec <Prashanth Govindarajan> sq
    6ca466fd0 <Prashanth Govindarajan> sq
    f1833c752 <Prashanth Govindarajan> sq
    cea217beb <Prashanth Govindarajan> Implement ChunkedArray, Column and Table
---
 csharp/src/Apache.Arrow/Arrays/Array.cs       |  14 ++++
 csharp/src/Apache.Arrow/Arrays/ArrayData.cs   |  13 +++
 csharp/src/Apache.Arrow/ChunkedArray.cs       |  91 +++++++++++++++++++++
 csharp/src/Apache.Arrow/Column.cs             |  74 +++++++++++++++++
 csharp/src/Apache.Arrow/Schema.cs             |  47 ++++++++++-
 csharp/src/Apache.Arrow/Table.cs              | 113 ++++++++++++++++++++++++++
 csharp/src/Apache.Arrow/Utility.cs            |  87 ++++++++++++++++++++
 csharp/test/Apache.Arrow.Tests/ColumnTests.cs |  58 +++++++++++++
 csharp/test/Apache.Arrow.Tests/TableTests.cs  |  83 +++++++++++++++++++
 9 files changed, 578 insertions(+), 2 deletions(-)

diff --git a/csharp/src/Apache.Arrow/Arrays/Array.cs b/csharp/src/Apache.Arrow/Arrays/Array.cs
index f9bd424..77794ab 100644
--- a/csharp/src/Apache.Arrow/Arrays/Array.cs
+++ b/csharp/src/Apache.Arrow/Arrays/Array.cs
@@ -60,6 +60,20 @@ namespace Apache.Arrow
             }
         }
 
+        public Array Slice(int offset, int length)
+        {
+            if (offset > Length)
+            {
+                throw new ArgumentException($"Offset {offset} cannot be greater than Length {Length} for Array.Slice");
+            }
+
+            length = Math.Min(Data.Length - offset, length);
+            offset += Data.Offset;
+
+            ArrayData newData = Data.Slice(offset, length);
+            return ArrowArrayFactory.BuildArray(newData) as Array;
+        }
+
         public void Dispose()
         {
             Dispose(true);
diff --git a/csharp/src/Apache.Arrow/Arrays/ArrayData.cs b/csharp/src/Apache.Arrow/Arrays/ArrayData.cs
index 76172f3..ef8ef03 100644
--- a/csharp/src/Apache.Arrow/Arrays/ArrayData.cs
+++ b/csharp/src/Apache.Arrow/Arrays/ArrayData.cs
@@ -73,5 +73,18 @@ namespace Apache.Arrow
                 }
             }
         }
+
+        public ArrayData Slice(int offset, int length)
+        {
+            if (offset > Length)
+            {
+                throw new ArgumentException($"Offset {offset} cannot be greater than Length {Length} for Array.Slice");
+            }
+
+            length = Math.Min(Length - offset, length);
+            offset += Offset;
+
+            return new ArrayData(DataType, length, -1, offset, Buffers, Children);
+        }
     }
 }
diff --git a/csharp/src/Apache.Arrow/ChunkedArray.cs b/csharp/src/Apache.Arrow/ChunkedArray.cs
new file mode 100644
index 0000000..5f25acf
--- /dev/null
+++ b/csharp/src/Apache.Arrow/ChunkedArray.cs
@@ -0,0 +1,91 @@
+// 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;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// A data structure to manage a list of primitive Array arrays logically as one large array
+    /// </summary>
+    public class ChunkedArray
+    {
+        private IList<Array> Arrays { get; }
+        public IArrowType DataType { get; }
+        public long Length { get; }
+        public long NullCount { get; }
+
+        public int ArrayCount
+        {
+            get => Arrays.Count;
+        }
+
+        public Array Array(int index) => Arrays[index];
+
+        public ChunkedArray(IList<Array> arrays)
+        {
+            Arrays = arrays ?? throw new ArgumentNullException(nameof(arrays));
+            if (arrays.Count < 1)
+            {
+                throw new ArgumentException($"Count must be at least 1. Got {arrays.Count} instead");
+            }
+            DataType = arrays[0].Data.DataType;
+            foreach (Array array in arrays)
+            {
+                Length += array.Length;
+                NullCount += array.NullCount;
+            }
+        }
+
+        public ChunkedArray(Array array) : this(new[] { array }) { }
+
+        public ChunkedArray Slice(long offset, long length)
+        {
+            if (offset >= Length)
+            {
+                throw new ArgumentException($"Index {offset} cannot be greater than the Column's Length {Length}");
+            }
+
+            int curArrayIndex = 0;
+            int numArrays = Arrays.Count;
+            while (curArrayIndex < numArrays && offset > Arrays[curArrayIndex].Length)
+            {
+                offset -= Arrays[curArrayIndex].Length;
+                curArrayIndex++;
+            }
+
+            IList<Array> newArrays = new List<Array>();
+            while (curArrayIndex < numArrays && length > 0)
+            {
+                newArrays.Add(Arrays[curArrayIndex].Slice((int)offset,
+                              length > Arrays[curArrayIndex].Length ? Arrays[curArrayIndex].Length : (int)length));
+                length -= Arrays[curArrayIndex].Length - offset;
+                offset = 0;
+                curArrayIndex++;
+            }
+            return new ChunkedArray(newArrays);
+        }
+
+        public ChunkedArray Slice(long offset)
+        {
+            return Slice(offset, Length - offset);
+        }
+
+        // TODO: Flatten for Structs
+    }
+}
diff --git a/csharp/src/Apache.Arrow/Column.cs b/csharp/src/Apache.Arrow/Column.cs
new file mode 100644
index 0000000..518e148
--- /dev/null
+++ b/csharp/src/Apache.Arrow/Column.cs
@@ -0,0 +1,74 @@
+// 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
+{
+    /// <summary>
+    /// A Column data structure that logically represents a column in a dataset
+    /// </summary>
+    public class Column
+    {
+        public Field Field { get;  }
+        private readonly ChunkedArray _columnArrays;
+        public ChunkedArray Data => _columnArrays;
+
+        public Column(Field field, IList<Array> arrays)
+        {
+            _columnArrays = new ChunkedArray(arrays);
+            Field = field;
+            if (!ValidateArrayDataTypes())
+            {
+                throw new ArgumentException($"{Field.DataType} must match {_columnArrays.DataType}");
+            }
+        }
+
+        private Column(Field field, ChunkedArray arrays)
+        {
+            Field = field;
+            _columnArrays = arrays;
+        }
+
+        public long Length => _columnArrays.Length;
+        public long NullCount => _columnArrays.NullCount;
+        public string Name => Field.Name;
+        public IArrowType Type => Field.DataType;
+
+        public Column Slice(int offset, int length)
+        {
+            return new Column(Field, _columnArrays.Slice(offset, length));
+        }
+
+        public Column Slice(int offset)
+        {
+            return new Column(Field, _columnArrays.Slice(offset));
+        }
+
+        private bool ValidateArrayDataTypes()
+        {
+            for (int i = 0; i < _columnArrays.ArrayCount; i++)
+            {
+                if (_columnArrays.Array(i).Data.DataType != Field.DataType)
+                {
+                    return false;
+                }
+            }
+            return true;
+        }
+    }
+}
diff --git a/csharp/src/Apache.Arrow/Schema.cs b/csharp/src/Apache.Arrow/Schema.cs
index 4255e26..25d7d80 100644
--- a/csharp/src/Apache.Arrow/Schema.cs
+++ b/csharp/src/Apache.Arrow/Schema.cs
@@ -21,7 +21,13 @@ namespace Apache.Arrow
 {
     public partial class Schema
     {
-        public IReadOnlyDictionary<string, Field> Fields { get; }
+        public IReadOnlyDictionary<string, Field> Fields
+        {
+            get => _fieldsDictionary;
+        }
+
+        private readonly Dictionary<string, Field> _fieldsDictionary;
+
         public IReadOnlyDictionary<string, string> Metadata { get; }
 
         public bool HasMetadata =>
@@ -40,7 +46,7 @@ namespace Apache.Arrow
 
             _fields = fields.ToList();
 
-            Fields = fields.ToDictionary(
+            _fieldsDictionary = fields.ToDictionary(
                 field => field.Name, field => field,
                 StringComparer.OrdinalIgnoreCase);
 
@@ -63,5 +69,42 @@ namespace Apache.Arrow
             return _fields.IndexOf(
                 _fields.Single(x => comparer.Compare(x.Name, name) == 0));
         }
+
+        public Schema RemoveField(int fieldIndex)
+        {
+            if (fieldIndex < 0 || fieldIndex >= _fields.Count)
+            {
+                throw new ArgumentException("Invalid fieldIndex", nameof(fieldIndex));
+            }
+
+            IList<Field> fields = Utility.DeleteListElement(_fields, fieldIndex);
+
+            return new Schema(fields, Metadata);
+        }
+
+        public Schema InsertField(int fieldIndex, Field newField)
+        {
+            newField = newField ?? throw new ArgumentNullException(nameof(newField));
+            if (fieldIndex < 0 || fieldIndex > _fields.Count)
+            {
+                throw new ArgumentException(nameof(fieldIndex), $"Invalid fieldIndex {fieldIndex} passed in to Schema.AddField");
+            }
+
+            IList<Field> fields = Utility.AddListElement(_fields, fieldIndex, newField);
+
+            return new Schema(fields, Metadata);
+        }
+
+        public Schema SetField(int fieldIndex, Field newField)
+        {
+            if (fieldIndex <0 || fieldIndex >= Fields.Count)
+            {
+                throw new ArgumentException($"Invalid fieldIndex {fieldIndex} passed in to Schema.SetColumn");
+            }
+
+            IList<Field> fields = Utility.SetListElement(_fields, fieldIndex, newField);
+
+            return new Schema(fields, Metadata);
+        }
     }
 }
diff --git a/csharp/src/Apache.Arrow/Table.cs b/csharp/src/Apache.Arrow/Table.cs
new file mode 100644
index 0000000..7116ff2
--- /dev/null
+++ b/csharp/src/Apache.Arrow/Table.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;
+
+namespace Apache.Arrow
+{
+    /// <summary>
+    /// A logical Table class to represent a dataset as a sequence of Columns
+    /// </summary>
+    public class Table
+    {
+        public Schema Schema { get; }
+        public long RowCount { get; }
+        public int ColumnCount { get; private set; }
+        public Column Column(int columnIndex) => _columns[columnIndex];
+
+        private readonly IList<Column> _columns;
+        public static Table TableFromRecordBatches(Schema schema, IList<RecordBatch> recordBatches)
+        {
+            int nBatches = recordBatches.Count;
+            int nColumns = schema.Fields.Count;
+
+            List<Column> columns = new List<Column>(nColumns);
+            List<Array> columnArrays = new List<Array>(nBatches);
+            for (int icol = 0; icol < nColumns; icol++)
+            {
+                for (int jj = 0; jj < nBatches; jj++)
+                {
+                    columnArrays.Add(recordBatches[jj].Column(icol) as Array);
+                }
+                columns.Add(new Arrow.Column(schema.GetFieldByIndex(icol), columnArrays));
+                columnArrays.Clear();
+            }
+
+            return new Table(schema, columns);
+        }
+
+        public Table(Schema schema, IList<Column> columns)
+        {
+            Schema = schema;
+            _columns = columns;
+            if (columns.Count > 0)
+            {
+                RowCount = columns[0].Length;
+                ColumnCount = columns.Count;
+            }
+        }
+
+        public Table()
+        {
+            Schema = new Schema.Builder().Build();
+            _columns = new List<Column>();
+        }
+
+        public Table RemoveColumn(int columnIndex)
+        {
+            Schema newSchema = Schema.RemoveField(columnIndex);
+            IList<Column> newColumns = Utility.DeleteListElement(_columns, columnIndex);
+            return new Table(newSchema, newColumns);
+        }
+
+        public Table InsertColumn(int columnIndex, Column column)
+        {
+            column = column ?? throw new ArgumentNullException(nameof(column));
+            if (columnIndex < 0 || columnIndex > _columns.Count)
+            {
+                throw new ArgumentException($"Invalid columnIndex {columnIndex} passed into Table.AddColumn");
+            }
+            if (column.Length != RowCount)
+            {
+                throw new ArgumentException($"Column's length {column.Length} must match Table's length {RowCount}");
+            }
+
+            Schema newSchema = Schema.InsertField(columnIndex, column.Field);
+            IList<Column> newColumns = Utility.AddListElement(_columns, columnIndex, column);
+            return new Table(newSchema, newColumns);
+        }
+
+        public Table SetColumn(int columnIndex, Column column)
+        {
+            column = column ?? throw new ArgumentNullException(nameof(column));
+            if (columnIndex < 0 || columnIndex >= ColumnCount)
+            {
+                throw new ArgumentException($"Invalid columnIndex {columnIndex} passed in to Table.SetColumn");
+            }
+
+            if (column.Length != RowCount)
+            {
+                throw new ArgumentException($"Column's length {column.Length} must match table's length {RowCount}");
+            }
+
+            Schema newSchema = Schema.SetField(columnIndex, column.Field);
+            IList<Column> newColumns = Utility.SetListElement(_columns, columnIndex, column);
+            return new Table(newSchema, newColumns);
+        }
+
+        // TODO: Flatten for Tables with Lists/Structs?
+    }
+}
diff --git a/csharp/src/Apache.Arrow/Utility.cs b/csharp/src/Apache.Arrow/Utility.cs
new file mode 100644
index 0000000..c4e5732
--- /dev/null
+++ b/csharp/src/Apache.Arrow/Utility.cs
@@ -0,0 +1,87 @@
+// 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.Flatbuf;
+using System;
+using System.Collections.Generic;
+using System.Text;
+
+namespace Apache.Arrow
+{
+    internal static class Utility
+    {
+        public static IList<T> DeleteListElement<T>(IList<T> values, int index)
+        {
+            if (index < 0 || index >= values.Count)
+            {
+                throw new ArgumentException("Invalid index", nameof(index));
+            }
+
+            List<T> newList = new List<T>(values.Count - 1);
+            for (int i = 0; i < index; i++)
+            {
+                newList.Add(values[i]);
+            }
+            for (int i = index + 1; i < values.Count; i++)
+            {
+                newList.Add(values[i]);
+            }
+
+            return newList;
+        }
+
+        public static IList<T> AddListElement<T>(IList<T> values, int index, T newElement)
+        {
+            if (index < 0 || index > values.Count)
+            {
+                throw new ArgumentException("Invalid index", nameof(index));
+            }
+
+            List<T> newList = new List<T>(values.Count + 1);
+            for (int i = 0; i < index; i++)
+            {
+                newList.Add(values[i]);
+            }
+            newList.Add(newElement);
+            for (int i = index; i < values.Count; i++)
+            {
+                newList.Add(values[i]);
+            }
+
+            return newList;
+        }
+
+        public static IList<T> SetListElement<T>(IList<T> values, int index, T newElement)
+        {
+            if (index < 0 || index >= values.Count)
+            {
+                throw new ArgumentException("Invalid index", nameof(index));
+            }
+
+            List<T> newList = new List<T>(values.Count);
+            for (int i = 0; i < index; i++)
+            {
+                newList.Add(values[i]);
+            }
+            newList.Add(newElement);
+            for (int i = index + 1; i < values.Count; i++)
+            {
+                newList.Add(values[i]);
+            }
+
+            return newList;
+        }
+    }
+}
diff --git a/csharp/test/Apache.Arrow.Tests/ColumnTests.cs b/csharp/test/Apache.Arrow.Tests/ColumnTests.cs
new file mode 100644
index 0000000..b90c681
--- /dev/null
+++ b/csharp/test/Apache.Arrow.Tests/ColumnTests.cs
@@ -0,0 +1,58 @@
+// 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.Linq;
+using Apache.Arrow.Types;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class ColumnTests
+    {
+        public static Array MakeIntArray(int length)
+        {
+            // The following should be improved once the ArrayBuilder PR goes in
+            var intBuilder = new ArrowBuffer.Builder<int>();
+            intBuilder.AppendRange(Enumerable.Range(0, length).Select(x => x));
+            ArrowBuffer buffer = intBuilder.Build();
+            ArrayData intData = new ArrayData(Int32Type.Default, length, 0, 0, new[] { ArrowBuffer.Empty, buffer });
+            Array intArray = ArrowArrayFactory.BuildArray(intData) as Array;
+            return intArray;
+        }
+
+        [Fact]
+        public void TestColumn()
+        {
+            Array intArray = MakeIntArray(10);
+            Array intArrayCopy = MakeIntArray(10);
+
+            Field field = new Field.Builder().Name("f0").DataType(Int32Type.Default).Build();
+            Column column = new Column(field, new[] { intArray, intArrayCopy });
+
+            Assert.True(column.Name == field.Name);
+            Assert.True(column.Field == field);
+            Assert.Equal(20, column.Length);
+            Assert.Equal(0, column.NullCount);
+            Assert.Equal(field.DataType, column.Type);
+            
+            Column slice5 = column.Slice(0, 5);
+            Assert.Equal(5, slice5.Length);
+            Column sliceFull = column.Slice(2);
+            Assert.Equal(18, sliceFull.Length);
+            Column sliceMore = column.Slice(0, 25);
+            Assert.Equal(20, sliceMore.Length);
+        }
+    }
+}
diff --git a/csharp/test/Apache.Arrow.Tests/TableTests.cs b/csharp/test/Apache.Arrow.Tests/TableTests.cs
new file mode 100644
index 0000000..b919bf3
--- /dev/null
+++ b/csharp/test/Apache.Arrow.Tests/TableTests.cs
@@ -0,0 +1,83 @@
+// 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;
+using Xunit;
+
+namespace Apache.Arrow.Tests
+{
+    public class TableTests
+    {
+        public static Table MakeTableWithOneColumnOfTwoIntArrays(int lengthOfEachArray)
+        {
+            Array intArray = ColumnTests.MakeIntArray(lengthOfEachArray);
+            Array intArrayCopy = ColumnTests.MakeIntArray(lengthOfEachArray);
+
+            Field field = new Field.Builder().Name("f0").DataType(Int32Type.Default).Build();
+            Schema s0 = new Schema.Builder().Field(field).Build();
+
+            Column column = new Column(field, new List<Array> { intArray, intArrayCopy });
+            Table table = new Table(s0, new List<Column> { column });
+            return table;
+        }
+
+        [Fact]
+        public void TestEmptyTable()
+        {
+            Table table = new Table();
+            Assert.Equal(0, table.ColumnCount);
+            Assert.Equal(0, table.RowCount);
+        }
+
+        [Fact]
+        public void TestTableBasics()
+        {
+            Table table = MakeTableWithOneColumnOfTwoIntArrays(10);
+            Assert.Equal(20, table.RowCount);
+            Assert.Equal(1, table.ColumnCount);
+        }
+
+        [Fact]
+        public void TestTableAddRemoveAndSetColumn()
+        {
+            Table table = MakeTableWithOneColumnOfTwoIntArrays(10);
+
+            Array nonEqualLengthIntArray = ColumnTests.MakeIntArray(10);
+            Field field1 = new Field.Builder().Name("f1").DataType(Int32Type.Default).Build();
+            Column nonEqualLengthColumn = new Column(field1, new[] { nonEqualLengthIntArray});
+            Assert.Throws<ArgumentException>(() => table.InsertColumn(-1, nonEqualLengthColumn));
+            Assert.Throws<ArgumentException>(() => table.InsertColumn(1, nonEqualLengthColumn));
+
+            Array equalLengthIntArray = ColumnTests.MakeIntArray(20);
+            Field field2 = new Field.Builder().Name("f2").DataType(Int32Type.Default).Build();
+            Column equalLengthColumn = new Column(field2, new[] { equalLengthIntArray});
+            Column existingColumn = table.Column(0);
+
+            Table newTable = table.InsertColumn(0, equalLengthColumn);
+            Assert.Equal(2, newTable.ColumnCount);
+            Assert.True(newTable.Column(0) == equalLengthColumn);
+            Assert.True(newTable.Column(1) == existingColumn);
+
+            newTable = newTable.RemoveColumn(1);
+            Assert.Equal(1, newTable.ColumnCount);
+            Assert.True(newTable.Column(0) == equalLengthColumn);
+
+            newTable = table.SetColumn(0, existingColumn);
+            Assert.True(newTable.Column(0) == existingColumn);
+        }
+    }
+}