You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "CurtHagenlocher (via GitHub)" <gi...@apache.org> on 2023/06/06 19:51:24 UTC

[GitHub] [arrow-adbc] CurtHagenlocher commented on a diff in pull request #697: feat(csharp): adding C# functionality

CurtHagenlocher commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1218708967


##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache - Backup.Arrow.Adbc.FlightSql.csproj:
##########
@@ -0,0 +1,17 @@
+<Project Sdk="Microsoft.NET.Sdk">

Review Comment:
   This file should probably be removed



##########
csharp/Directory.Build.props:
##########
@@ -0,0 +1,60 @@
+<!--

Review Comment:
   Should there be a .sln file in the root like there is for arrow/csharp?



##########
csharp/Directory.Build.props:
##########
@@ -0,0 +1,60 @@
+<!--
+  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.
+-->
+
+<Project>
+
+  <!-- Common repo directories -->
+  <PropertyGroup>
+    <RepoRoot>$(MSBuildThisFileDirectory)../</RepoRoot>
+    <CSharpDir>$(MSBuildThisFileDirectory)</CSharpDir>
+    <BaseOutputPath>$(CSharpDir)/artifacts/$(MSBuildProjectName)</BaseOutputPath>
+  </PropertyGroup>
+
+  <!-- AssemblyInfo properties -->
+  <PropertyGroup>
+    <Product>Apache Arrow ADBC library</Product>
+    <Copyright>Copyright 2016-2019 The Apache Software Foundation</Copyright>

Review Comment:
   2023? Or 2022-2023? (The root has a file which says just 2022.)



##########
csharp/Directory.Build.props:
##########
@@ -0,0 +1,60 @@
+<!--
+  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.
+-->
+
+<Project>
+
+  <!-- Common repo directories -->
+  <PropertyGroup>
+    <RepoRoot>$(MSBuildThisFileDirectory)../</RepoRoot>
+    <CSharpDir>$(MSBuildThisFileDirectory)</CSharpDir>
+    <BaseOutputPath>$(CSharpDir)/artifacts/$(MSBuildProjectName)</BaseOutputPath>
+  </PropertyGroup>
+
+  <!-- AssemblyInfo properties -->
+  <PropertyGroup>
+    <Product>Apache Arrow ADBC library</Product>
+    <Copyright>Copyright 2016-2019 The Apache Software Foundation</Copyright>
+    <Company>The Apache Software Foundation</Company>
+    <Version>0.1.0</Version>

Review Comment:
   Should this reflect the release version of ADBC (which will next be 0.5, I think?)



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlDatabase.cs:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.Adbc.Core;
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// A Flight SQL implementation of <see cref="AdbcDatabase"/>.
+    /// </summary>
+    public class FlightSqlDatabase : AdbcDatabase
+    {
+        private readonly Dictionary<string, string> _metadata;
+
+        public FlightSqlDatabase(Dictionary<string, string> metadata)
+        {
+            _metadata = metadata;
+        }
+
+        public override AdbcConnection Connect(Dictionary<string, string> options)
+        {
+            if(options == null) throw new ArgumentNullException("options");

Review Comment:
   nit: it would be good to format consistently "if(" vs "if (".



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlDatabase.cs:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.Adbc.Core;
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// A Flight SQL implementation of <see cref="AdbcDatabase"/>.
+    /// </summary>
+    public class FlightSqlDatabase : AdbcDatabase
+    {
+        private readonly Dictionary<string, string> _metadata;
+
+        public FlightSqlDatabase(Dictionary<string, string> metadata)
+        {
+            _metadata = metadata;
+        }
+
+        public override AdbcConnection Connect(Dictionary<string, string> options)
+        {
+            if(options == null) throw new ArgumentNullException("options");
+
+            if (!options.ContainsKey(FlightSqlParameters.ServerAddress))

Review Comment:
   Consider using TryGetValue to save one lookup.



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlConnection.cs:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.Adbc.Core;
+using Apache.Arrow.Flight.Client;
+using Grpc.Core;
+using Grpc.Net.Client;
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// A Flight SQL implementation of <see cref="AdbcConnection"/>.
+    /// </summary>
+    public class FlightSqlConnection : AdbcConnection
+    {
+        private FlightClient _flightClientInternal = null;
+        private readonly Dictionary<string, string> _metadata;
+
+        private Metadata headers = null;
+
+        public FlightSqlConnection(Dictionary<string, string> metadata)

Review Comment:
   Should this be e.g. an IReadOnlyDictionary in order to telegraph the intent the caller shouldn't mutate the dict? Same applies to other classes which take a dictionary on construction and don't expect it to change.



##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcDatabase.cs:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.Adbc.Core
+{
+    /// <summary>
+    /// Clients first initialize a database, then create a connection. This gives the implementation a place to initialize and own any common connection state.  For example, in-memory databases can place ownership of the actual database in this object.

Review Comment:
   nit: some long doc comments are single-line and some are multi-line. It would be nice to have a more consistent treatment.



##########
csharp/src/Apache.Arrow.Adbc/Core/PartitionDescriptor.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;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public struct PartitionDescriptor : IEquatable<PartitionDescriptor>
+    {
+        readonly byte[] _descriptor;
+
+        public PartitionDescriptor(byte[] descriptor)
+        {
+            _descriptor = descriptor;
+        }
+
+        public override bool Equals(object obj)
+        {
+            PartitionDescriptor? other = obj as PartitionDescriptor?;
+            return other != null && Equals(other.Value);
+        }
+
+        public bool Equals(PartitionDescriptor other)
+        {
+            if (_descriptor.Length != other._descriptor.Length)
+            {
+                return false;
+            }
+            for (int i = 0; i < _descriptor.Length; i++)
+            {
+                if (_descriptor[i] != other._descriptor[i])
+                {
+                    return false;
+                }
+            }
+            return true;
+        }
+
+        public override int GetHashCode()
+        {
+            return base.GetHashCode();

Review Comment:
   This should be a real implementation or it won't satisfy the IEquatable contract.



##########
csharp/src/Apache.Arrow.Adbc/Core/StandardSchemas.cs:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// The standard schemas
+    /// </summary>
+    public sealed class StandardSchemas
+    {
+        private StandardSchemas()

Review Comment:
   This can't be a static class?



##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.Linq;
+using System.Text.RegularExpressions;
+using System.Threading.Tasks;
+using Apache.Arrow.C;
+using Apache.Arrow.Ipc;
+using Apache.Arrow.Types;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// Statements may represent queries or prepared statements. Statements may be used multiple times and can be reconfigured (e.g. they can be reused to execute multiple different queries).
+    /// </summary>
+    public abstract class AdbcStatement : IDisposable
+    {
+        public AdbcStatement()
+        {
+            Timeout = 30;
+        }
+
+        /// <summary>
+        /// Gets or sets a SQL query to be executed on this statement.
+        /// </summary>
+        public virtual string SqlQuery { get; set; }
+
+        /// <summary>
+        /// Gets or sets the Substrait plan.
+        /// </summary>
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind(RecordBatch batch,Schema schema)
+        {
+            throw AdbcException.NotImplemented("Statement does not support Bind");
+        }
+
+        /// <summary>
+        /// Executes the statement and returns a tuple containing the number of records and the <see cref="IArrowArrayStream"/>..
+        /// </summary>
+        /// <returns>A <see cref="ValueTuple"/> where the first item is the number of records and the second is the <see cref="IArrowArrayStream"/>.</returns>
+        public abstract QueryResult ExecuteQuery();
+
+        /// <summary>
+        /// Executes the statement and returns a tuple containing the number of records and the <see cref="IArrowArrayStream"/>..
+        /// </summary>
+        /// <returns>A <see cref="ValueTuple"/> where the first item is the number of records and the second is the <see cref="IArrowArrayStream"/>.</returns>
+        public virtual async ValueTask<QueryResult> ExecuteQueryAsync()
+        {
+            return await Task.Run(() => ExecuteQuery());
+        }
+
+        /// <summary>
+        /// Executes an update command and returns the number of records effected.
+        /// </summary>
+        /// <returns></returns>
+        /// <exception cref="NotImplementedException"></exception>
+        public abstract UpdateResult ExecuteUpdate();
+
+        // <summary>
+        /// Executes an update command and returns the number of records effected.
+        /// </summary>
+        /// <returns></returns>
+        /// <exception cref="NotImplementedException"></exception>
+        public virtual async Task<UpdateResult> ExecuteUpdateAsync()
+        {
+            return await Task.Run(() => ExecuteUpdate());
+        }
+
+        /// <summary>
+        /// Timeout (in seconds) for statement execution.
+        /// </summary>
+        /// <remarks>The default is 30 seconds.</remarks>
+        public virtual int Timeout { get; set; }

Review Comment:
   I know this is consistent with ADO.NET, but I feel like more modern APIs will typically remove ambiguity by using a TimeSpan instead of an "int" to describe durations.



##########
csharp/src/Apache.Arrow.Adbc/Core/StandardSchemas.cs:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// The standard schemas
+    /// </summary>
+    public sealed class StandardSchemas
+    {
+        private StandardSchemas()
+        {
+            throw new InvalidOperationException("Do not instantiate this class");
+        }
+
+        private static readonly ArrowType INT16 = new Int16Type();

Review Comment:
   I think these can be removed now that e.g. Int16Type.Default is being used



##########
csharp/src/Apache.Arrow.Adbc/Interop/NativePointer.cs:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.Runtime.InteropServices;
+
+namespace Apache.Arrow.Adbc.Interop
+{
+    internal readonly struct NativeDelegate<T>

Review Comment:
   Consider renaming the file to match the class.
   
   If you look at a recent in-progress change for Arrow (https://github.com/apache/arrow/pull/35810) there's some pertinent work and discussion related to NativeDelegate:
   
   1) It's not needed for .NET >= 5.0 due to the ability to declare a function as [UnmanagedCallersOnly]
   2) The native delegates have been typed somewhat incorrectly, and in particular would produce the wrong outcome when building a 32-bit version for non-Windows platforms.
   
   The proposed solutions are
   1) Only use NativeDelegate when targeting older .NET
   2) Make the actual delegate fields in the structs be private and give them different calling conventions depending on the version of .NET.
   
   That said, the change is still in progress and not final, and there's no reason the same change can't be made here after the initial commit.



##########
csharp/src/Apache.Arrow.Adbc/Core/StandardSchemas.cs:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// The standard schemas
+    /// </summary>
+    public sealed class StandardSchemas
+    {
+        private StandardSchemas()
+        {
+            throw new InvalidOperationException("Do not instantiate this class");
+        }
+
+        private static readonly ArrowType INT16 = new Int16Type();
+        private static readonly ArrowType INT32 = new Int32Type();
+        private static readonly ArrowType INT64 = new Int64Type();
+        private static readonly ArrowType UINT32 = new UInt32Type();
+
+        /// <summary>
+        /// The schema of the result set of <see cref="AdbcConnection.GetInfo(int[])"/>}.
+        /// </summary>
+        public static readonly Schema GetInfoSchema =
+            new Schema(
+                new List<Field>()
+                {
+                    new Field("info_name", UInt32Type.Default, false),
+                    new Field(
+                        "info_value",
+                        new UnionType(
+                            new List<Field>()
+                            {
+                                new Field("string_value", StringType.Default, true),
+                                new Field("bool_value", BooleanType.Default, true),
+                                new Field("int64_value", Int64Type.Default, true),
+                                new Field("int32_bitmask", Int32Type.Default, true),
+                                new Field(
+                                    "string_list",
+                                    new ListType(
+                                        new Field("item", StringType.Default, true)
+                                    ),
+                                    false
+                                ),
+                                new Field(
+                                    "int32_to_int32_list_map",
+                                    new ListType(
+                                        new Field("entries", new StructType(
+                                            new List<Field>()
+                                            {
+                                                new Field("key", Int32Type.Default, false),
+                                                new Field("value", Int32Type.Default, true),
+                                            }
+                                            ), false)
+                                    ),
+                                    true
+                                )
+                            }, 
+                            // TBD if this line is the best approach but its a good one-liner
+                            new int[] {0, 1, 2, 3, 4, 5}.SelectMany(BitConverter.GetBytes).ToArray(),
+                            UnionMode.Dense),
+                        true)
+                },
+                metadata: null
+        );
+
+        public static readonly Schema TableTypesSchema = new Schema(
+            new List<Field>()
+            {
+                new Field("table_type", StringType.Default, false)
+            },
+            metadata: null
+        );
+
+        public static readonly List<Field> UsageSchema = new List<Field>()
+        {
+            new Field("fk_catalog", StringType.Default, true),
+            new Field("fk_db_schema", StringType.Default, true),
+            new Field("fk_table", StringType.Default, false),
+            new Field("fk_column_name", StringType.Default, false)
+        };
+
+        public static readonly List<Field> ConstraintSchema = new List<Field>()
+        {
+            new Field("constraint_name", StringType.Default, false),
+            new Field("constraint_type", StringType.Default, false),
+            new Field("constraint_column_usage",
+                new ListType(
+                    new Field("item", StringType.Default, true)
+                ),
+                false
+            ),
+            new Field("constraint_column_usage",
+                new ListType(
+                    new Field("item", new StructType(UsageSchema), true)
+                ),
+                false
+            ),
+        };
+
+        public static readonly List<Field> ColumnSchema =
+            new List<Field>()
+            {
+                new Field("column_name", StringType.Default, false),
+                new Field("ordinal_position", Int32Type.Default, true),
+                new Field("remarks", StringType.Default, true),
+                new Field("xdbc_data_type", Int16Type.Default, true),
+                new Field("xdbc_type_name", StringType.Default, true),
+                new Field("xdbc_column_size", Int32Type.Default, true),
+                new Field("xdbc_decimal_digits", Int16Type.Default, true),
+                new Field("xdbc_num_prec_radix", Int16Type.Default, true),
+                new Field("xdbc_nullable", Int16Type.Default, true),
+                new Field("xdbc_column_def", StringType.Default, true),
+                new Field("xdbc_sql_data_type", Int16Type.Default, true),
+                new Field("xdbc_datetime_sub", Int16Type.Default, true),
+                new Field("xdbc_char_octet_length", Int32Type.Default, true),
+                new Field("xdbc_is_nullable", StringType.Default, true),
+                new Field("xdbc_scope_catalog", StringType.Default, true),
+                new Field("xdbc_scope_schema", StringType.Default, true),
+                new Field("xdbc_scope_table", StringType.Default, true),
+                new Field("xdbc_is_autoincrement", StringType.Default, true),
+                new Field("xdbc_is_generatedcolumn", StringType.Default, true)
+            };
+
+        public static readonly List<Field> TableSchema = new List<Field>() {
+          new Field("table_name", StringType.Default, false, null),
+          new Field("table_type", StringType.Default, false, null),
+          new Field(
+              "table_columns",
+              new ListType(
+                new Field("item", new StructType(ColumnSchema), true)
+              ),
+              false
+          ),
+          new Field(
+              "table_constraints",
+              new ListType(
+                new Field("item", new StructType(ConstraintSchema), true)
+              ),
+              false
+          )
+        };
+
+        public static readonly List<Field> DbSchemaSchema = new List<Field>()
+        {
+            new Field("db_schema_name", StringType.Default, false, null),
+            new Field(
+                "db_schema_tables",
+                new ListType(
+                    new Field("item", new StructType(TableSchema), true)
+                ),
+                false
+            )
+        };
+
+        public static readonly Schema GET_OBJECTS_SCHEMA = new Schema(

Review Comment:
   Why is this one yelling at me? :P



##########
csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/ConnectionTests.cs:
##########
@@ -0,0 +1,138 @@
+/*
+* 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.IO;
+using System.Linq;
+using System.Text.Json;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Adbc.Tests;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Moq;
+
+namespace Apache.Arrow.Adbc.FlightSql.Tests
+{
+    /// <summary>
+    /// Abstract class for the ADBC connection tests.
+    /// </summary>
+    [TestClass]
+    public class ConnectionTests
+    {
+        /// <summary>
+        /// Validates if the driver behaves as it should with missing values and parsing mock results.
+        /// </summary>
+        [TestMethod]
+        public void CanMockDriverConnect()
+        {
+            Mock<FlightSqlStatement> mockFlightSqlStatement = GetMockSqlStatement();
+
+            FlightSqlDatabase db = new FlightSqlDatabase(new Dictionary<string, string>());
+
+            Assert.ThrowsException<ArgumentNullException>(() => db.Connect(null));
+
+            Assert.ThrowsException<ArgumentException>(() => db.Connect(new Dictionary<string, string>()));
+
+            QueryResult queryResult = mockFlightSqlStatement.Object.ExecuteQuery();
+
+            Adbc.Tests.ConnectionTests.CanDriverConnect(queryResult, 50);
+        }
+
+        /// <summary>
+        /// Validates if the driver can connect to a live server and parse the results.
+        /// </summary>
+        [TestMethod]
+        public void CanDriverConnect()
+        {
+            FlightSqlTestConfiguration flightSqlTestConfiguration = GetFlightSqlTestConfiguration();
+
+            Dictionary<string, string> parameters = new Dictionary<string, string>
+            {
+                { FlightSqlParameters.ServerAddress, flightSqlTestConfiguration.ServerAddress },
+                { FlightSqlParameters.RoutingTag, flightSqlTestConfiguration.RoutingTag },
+                { FlightSqlParameters.RoutingQueue, flightSqlTestConfiguration.RoutingQueue },
+                { FlightSqlParameters.Authorization, flightSqlTestConfiguration.Authorization}
+            };
+
+            Dictionary<string, string> options = new Dictionary<string, string>()
+            {
+                { FlightSqlParameters.ServerAddress, flightSqlTestConfiguration.ServerAddress },
+            };
+
+            FlightSqlDriver flightSqlDriver = new FlightSqlDriver();
+            FlightSqlDatabase flightSqlDatabase = flightSqlDriver.Open(parameters) as FlightSqlDatabase;
+            FlightSqlConnection connection = flightSqlDatabase.Connect(options) as FlightSqlConnection;
+            FlightSqlStatement statement = connection.CreateStatement() as FlightSqlStatement;
+
+            statement.SqlQuery = flightSqlTestConfiguration.Query;
+            QueryResult queryResult = statement.ExecuteQuery();
+
+            Adbc.Tests.ConnectionTests.CanDriverConnect(queryResult, flightSqlTestConfiguration.ExpectedResultsCount);
+        }
+        /// <summary>

Review Comment:
   nit: insert blank line?



##########
csharp/src/Apache.Arrow.Adbc/Extensions/MarshalExtensions.netstandard.cs:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.
+ */
+
+#if NETSTANDARD

Review Comment:
   Consider removing via the project file like the Arrow project does.



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