You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "davidhcoe (via GitHub)" <gi...@apache.org> on 2023/05/22 18:37:03 UTC

[GitHub] [arrow-adbc] davidhcoe opened a new pull request, #697: adding C# functionality

davidhcoe opened a new pull request, #697:
URL: https://github.com/apache/arrow-adbc/pull/697

   Introduces C# functionality for ADBC, including  base classes for
   
   - AdbcDriver
   - AdbcDatabase
   - AdbcConnection
   - AdbcStatement
   
   Also adds a FlightSql implementation
   
   Currently tied to the submodule that is awaiting https://github.com/apache/arrow/pull/35496


-- 
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-adbc] lidavidm commented on pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1579351411

   Also note the pre-commit CI failures (mostly: trim trailing whitespace, and make sure the Apache license header is added to new files), then I think it's ready to merge (can we start filing follow-up issues?)


-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220837533


##########
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:
   fixed



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1221591352


##########
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:
   done



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1204679363


##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlStatement.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.Threading.Tasks;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Flight;
+using Grpc.Core;
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// A Flight SQL implementation of <see cref="AdbcStatement"/>.
+    /// </summary>
+    public class FlightSqlStatement : AdbcStatement
+    {
+        private FlightSqlConnection flightSqlConnection;
+        
+        public FlightSqlStatement(FlightSqlConnection flightSqlConnection)
+        {
+            this.flightSqlConnection = flightSqlConnection;
+        }
+
+        public override async ValueTask<QueryResult> ExecuteQueryAsync()
+        {
+            FlightInfo info = await GetInfo(this.SqlQuery, this.flightSqlConnection.Metadata);
+
+            return new QueryResult(info.TotalRecords, new FlightSqlResult(this.flightSqlConnection, info));
+        }
+
+        public override QueryResult ExecuteQuery()
+        {
+            return ExecuteQueryAsync().Result;
+        }
+
+        public override UpdateResult ExecuteUpdate()
+        {
+            throw new NotImplementedException();
+        }
+
+        public async ValueTask<FlightInfo> GetInfo(string query, Metadata headers)
+        {
+            FlightDescriptor commandDescripter = FlightDescriptor.CreateCommandDescriptor(query);
+
+            return await this.flightSqlConnection.FlightClient.GetInfo(commandDescripter, headers).ResponseAsync;
+        }
+
+        /// <summary>
+        /// Gets a value from the Arrow array at the specified index using the Arrow field for metadata.
+        /// </summary>
+        /// <param name="arrowArray"></param>
+        /// <param name="field"></param>
+        /// <param name="index"></param>
+        /// <returns></returns>
+        public override object GetValue(IArrowArray arrowArray, Field field, int index)

Review Comment:
   Similar to the explanation for ConvertArrowType, it's here for each driver to interpret the value. Again, in Snowflake, if the type is an Int64Array, but the logical type is _TIME_, then some special calculations need to be performed to convert that long value to a DateTime value in C#. There are several others, but these types of nuances are why I left it up to the drivers to implement.



-- 
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-adbc] lidavidm commented on pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1580697873

   It looks like there's still whitespace errors: https://github.com/apache/arrow-adbc/actions/runs/5196453814/jobs/9377170741?pr=697
   
   if you set up [pre-commit](https://pre-commit.com/), you can run `pre-commit` in the repo root then commit 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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220837090


##########
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:
   done



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1218379780


##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj:
##########
@@ -0,0 +1,20 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AssemblyName>$(MSBuildProjectName)</AssemblyName>
+    <RootNamespace>Apache.Arrow.Adbc.FlightSql</RootNamespace>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <PackageReference Include="Apache.Arrow.Flight" Version="12.0.0" />
+    <PackageReference Include="Grpc.Net.Client.Web" Version="2.53.0" />

Review Comment:
   good catch. this was left over from getting things to work with .NET Standard. removed



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1204651377


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.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; }
+
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind()
+        {
+            throw new NotImplementedException();
+        }
+
+        /// <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; }
+
+        /// <summary>
+        /// Execute a result set-generating query and get a list of partitions of the result set.
+        /// </summary>
+        /// <returns><see cref="PartitionedResult"/></returns>
+        public virtual PartitionedResult ExecutePartitioned()
+        {
+            throw AdbcException.NotImplemented("Statement does not support executePartitioned");
+        }
+
+        public virtual Schema GetParameterSchema()
+        {
+            throw AdbcException.NotImplemented("Statement does not support GetParameterSchema");
+        }
+
+        public virtual void Prepare()
+        {
+            throw AdbcException.NotImplemented("Statement does not support Prepare");
+        }
+
+        public virtual void Dispose()
+        {
+        }
+
+        /// <summary>
+        /// Gets the .NET type based on the Arrow field metadata
+        /// </summary>
+        /// <param name="f"></param>
+        /// <returns></returns>
+        public virtual Type ConvertArrowType(Field f)

Review Comment:
   I put it there so that each driver can provide its own implementation. For example, in Snowflake, if a Field.DataType.TypeId is ArrowTypeId.Int8, that seems like it would be an sbyte in C#. However, for Snowflake in particular, the Field metadata needs to be examined to determine if the logical type is a _FIXED_. If so, the _scale_ metadata is also examined. If the scale is zero, then the correct type is _long_. However, if scale != 0, then a _decimal_ type is returned. 



-- 
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-adbc] eerhardt commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1214703073


##########
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 Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Adbc.Tests;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Moq;
+using Newtonsoft.Json;

Review Comment:
   (minor request) Can we use `System.Text.Json` instead of `Newtonsoft.Json`? I know this is just tests, but it would be good to use the stuff in the box, if possible.



##########
csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/.gitignore:
##########
@@ -0,0 +1 @@
+*.pass

Review Comment:
   Do we need this? It is in the csharp/.gitignore.



##########
csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj:
##########
@@ -0,0 +1,13 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <Version>0.1.0</Version>

Review Comment:
   ```suggestion
   ```
   
   We will want to read this from the root Directory.Build.props.



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj:
##########
@@ -0,0 +1,20 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AssemblyName>$(MSBuildProjectName)</AssemblyName>
+    <RootNamespace>Apache.Arrow.Adbc.FlightSql</RootNamespace>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <PackageReference Include="Apache.Arrow.Flight" Version="12.0.0" />
+    <PackageReference Include="Grpc.Net.Client.Web" Version="2.53.0" />

Review Comment:
   ```suggestion
       <PackageReference Include="Grpc.Net.Client" Version="2.53.0" />
   ```
   
   I assume you don't really need the `.Web` package. If you do, why?



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj:
##########
@@ -0,0 +1,20 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AssemblyName>$(MSBuildProjectName)</AssemblyName>
+    <RootNamespace>Apache.Arrow.Adbc.FlightSql</RootNamespace>

Review Comment:
   ```suggestion
   ```
   
   These aren't needed - they are by default set to the project name.



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;
+
+        /// <summary>
+        /// The length of each corresponding entry in partitions.
+        /// </summary>
+        public int* partition_lengths;

Review Comment:
   ```suggestion
           public nuint* partition_lengths;
   
   ```



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;
+
+        /// <summary>
+        /// The length of each corresponding entry in partitions.
+        /// </summary>
+        public int* partition_lengths;
+
+        /// <summary>
+        /// Opaque implementation-defined state.
+        /// This field is NULLPTR iff the connection is unintialized/freed.
+        /// </summary>
+        public void* private_data;
+
+        /// <summary>
+        /// Release the contained partitions.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        /// </summary>
+        public delegate* unmanaged[Stdcall]<NativeAdbcPartitions*, void> release; 
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcError
+    {
+        /// <summary>
+        /// The error message.
+        /// </summary>
+        public char* message;

Review Comment:
   ```suggestion
           public byte* message;
   ```



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj:
##########
@@ -0,0 +1,20 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AssemblyName>$(MSBuildProjectName)</AssemblyName>
+    <RootNamespace>Apache.Arrow.Adbc.FlightSql</RootNamespace>
+    <Version>0.1.0</Version>

Review Comment:
   ```suggestion
   ```



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);

Review Comment:
   Is this right? Why are we using COM memory? We should be using `Marshal.AllocHGlobal` and `Marshal.FreeHGlobal`.



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj:
##########
@@ -0,0 +1,20 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AssemblyName>$(MSBuildProjectName)</AssemblyName>
+    <RootNamespace>Apache.Arrow.Adbc.FlightSql</RootNamespace>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <PackageReference Include="Apache.Arrow.Flight" Version="12.0.0" />
+    <PackageReference Include="Grpc.Net.Client.Web" Version="2.53.0" />
+    <PackageReference Include="System.Net.Http.WinHttpHandler" Version="7.0.0" />

Review Comment:
   ```suggestion
       <PackageReference Include="System.Net.Http.WinHttpHandler" Version="7.0.0" Condition="'$(TargetFrameworkIdentifier)' == '.NETStandard'" />
   ```
   
   We don't want to bring this dependency in on `net6.0`+.



##########
csharp/src/Apache.Arrow.Adbc/Extensions/MarshalExtensions.netstandard.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.
+ */
+
+#if NETSTANDARD
+using System;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Text;
+
+namespace Apache.Arrow.Adbc.Extensions
+{
+    public static class MarshalExtensions
+    {
+        public static unsafe string PtrToStringUTF8(IntPtr intPtr)
+        {
+            if (intPtr == null) 
+            {
+                return null;
+            }
+
+            unsafe
+            {
+                byte* source = (byte*)intPtr;
+                int nbBytes =  source[0];

Review Comment:
   Is this right? 
   
   1. These strings have the length at the beginning? Is there some guarantee about that?
   2. Since `source` is a `byte*`, `source[0]` returns a `byte`, which means the `nbBytes` can only be between 0-255. These strings can't be longer than that?
   
   I would have assumed we would need to `strlen` here - like the real Marshal.PtrToStringUTF8 does - https://github.com/dotnet/runtime/blob/7febca6802545d8c96e386187d536bc100d40337/src/libraries/System.Private.CoreLib/src/System/Runtime/InteropServices/Marshal.cs#L76



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;

Review Comment:
   Do these need to be filled out?



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;
+
+        /// <summary>
+        /// The length of each corresponding entry in partitions.
+        /// </summary>
+        public int* partition_lengths;
+
+        /// <summary>
+        /// Opaque implementation-defined state.
+        /// This field is NULLPTR iff the connection is unintialized/freed.
+        /// </summary>
+        public void* private_data;
+
+        /// <summary>
+        /// Release the contained partitions.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        /// </summary>
+        public delegate* unmanaged[Stdcall]<NativeAdbcPartitions*, void> release; 
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcError
+    {
+        /// <summary>
+        /// The error message.
+        /// </summary>
+        public char* message;
+
+        /// <summary>
+        /// A vendor-specific error code, if applicable.
+        /// </summary>
+        public int vendor_code;
+
+        /// <summary>
+        /// A SQLSTATE error code, if provided, as defined by the
+        ///   SQL:2003 standard.  If not set, it should be set to
+        ///   "\0\0\0\0\0".
+        ///</summary>
+        public char sqlstate0;
+        public char sqlstate1;
+        public char sqlstate2;
+        public char sqlstate3;
+        public char sqlstate4;

Review Comment:
   ```suggestion
           public byte sqlstate0;
           public byte sqlstate1;
           public byte sqlstate2;
           public byte sqlstate3;
           public byte sqlstate4;
   ```



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;

Review Comment:
   ```suggestion
           public byte** partitions;
   ```



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;
+
+        /// <summary>
+        /// The length of each corresponding entry in partitions.
+        /// </summary>
+        public int* partition_lengths;
+
+        /// <summary>
+        /// Opaque implementation-defined state.
+        /// This field is NULLPTR iff the connection is unintialized/freed.
+        /// </summary>
+        public void* private_data;
+
+        /// <summary>
+        /// Release the contained partitions.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        /// </summary>
+        public delegate* unmanaged[Stdcall]<NativeAdbcPartitions*, void> release; 
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcError
+    {
+        /// <summary>
+        /// The error message.
+        /// </summary>
+        public char* message;
+
+        /// <summary>
+        /// A vendor-specific error code, if applicable.
+        /// </summary>
+        public int vendor_code;
+
+        /// <summary>
+        /// A SQLSTATE error code, if provided, as defined by the
+        ///   SQL:2003 standard.  If not set, it should be set to
+        ///   "\0\0\0\0\0".
+        ///</summary>
+        public char sqlstate0;
+        public char sqlstate1;
+        public char sqlstate2;
+        public char sqlstate3;
+        public char sqlstate4;

Review Comment:
   I haven't checked all of structs myself against the header file. It would be good to double-check all the types used in C# correspond to the type used in `adbc.h`.



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1202904642


##########
csharp/src/Apache.Arrow.Adbc/Extensions/MarshalExtensions.netstandard.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.
+ */
+
+#if NETSTANDARD
+using System;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Text;
+
+namespace Apache.Arrow.Adbc.Extensions 
+{
+    public static class MarshalExtensions
+    {
+        public static unsafe string PtrToStringUTF8(IntPtr intPtr)
+        {
+            if (intPtr == null) // IsNullOrWin32Atom(intPtr))

Review Comment:
   comment removed, but yes



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1206210623


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcException.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 Apache.Arrow.Adbc.Core;
+
+/// <summary>
+/// The root exception when working with Adbc drivers.
+/// </summary>
+public class AdbcException : Exception
+{
+    private AdbcStatusCode statusCode = AdbcStatusCode.UnknownError;
+
+    public AdbcException()
+    {
+    }
+
+    public AdbcException(string message)
+        : base(message)
+    {
+    }
+
+    public AdbcException(string message, AdbcStatusCode statusCode)
+        : base(message)
+    {
+        this.statusCode = statusCode;
+    }
+
+    public AdbcException(string message, AdbcStatusCode statusCode, Exception innerException)
+       : base(message, innerException)
+    {
+    }
+
+    public AdbcException(string message, Exception innerException)
+        : base(message, innerException)
+    {
+    }
+
+    public static AdbcException NotImplemented(string message)
+    {
+        return new AdbcException(message, AdbcStatusCode.NotImplemented);
+    }
+
+    //
+    // Summary:
+    //     For database providers which support it, contains a standard SQL 5-character
+    //     return code indicating the success or failure of the database operation. The
+    //     first 2 characters represent the class of the return code (e.g. error, success),
+    //     while the last 3 characters represent the subclass, allowing detection of error
+    //     scenarios in a database-portable way.
+    //     For database providers which don't support it, or for inapplicable error scenarios,
+    //     contains null.
+    //
+    // Returns:
+    //     A standard SQL 5-character return code, or null.
+    public virtual string SqlState
+    {
+        get { return null; }
+        protected set { throw new NotImplementedException(); }

Review Comment:
   addressing in next push



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1201093734


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcConnection.cs:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.Ipc;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// Provides methods for query execution, managing prepared statements, using transactions, and so on.
+    /// </summary>
+    public abstract class AdbcConnection : IDisposable

Review Comment:
   There are different schools of thought to that. We do have an ADO.NET implementation that sits on top of this. However, ADO.NET is row-oriented, so it felt important to stick to Arrow libraries. We can also publish the ADO.NET one if that’s desired.



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1215027750


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;
+
+        /// <summary>
+        /// The length of each corresponding entry in partitions.
+        /// </summary>
+        public int* partition_lengths;
+
+        /// <summary>
+        /// Opaque implementation-defined state.
+        /// This field is NULLPTR iff the connection is unintialized/freed.
+        /// </summary>
+        public void* private_data;
+
+        /// <summary>
+        /// Release the contained partitions.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        /// </summary>
+        public delegate* unmanaged[Stdcall]<NativeAdbcPartitions*, void> release; 
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcError
+    {
+        /// <summary>
+        /// The error message.
+        /// </summary>
+        public char* message;
+
+        /// <summary>
+        /// A vendor-specific error code, if applicable.
+        /// </summary>
+        public int vendor_code;
+
+        /// <summary>
+        /// A SQLSTATE error code, if provided, as defined by the
+        ///   SQL:2003 standard.  If not set, it should be set to
+        ///   "\0\0\0\0\0".
+        ///</summary>
+        public char sqlstate0;
+        public char sqlstate1;
+        public char sqlstate2;
+        public char sqlstate3;
+        public char sqlstate4;

Review Comment:
   resolved in latest push. leaving this open to confirm the other types are 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-adbc] lidavidm commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1201016060


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcException.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 Apache.Arrow.Adbc.Core;
+
+/// <summary>
+/// The root exception when working with Adbc drivers.
+/// </summary>
+public class AdbcException : Exception
+{
+    private AdbcStatusCode statusCode = AdbcStatusCode.UnknownError;
+
+    public AdbcException()
+    {
+    }
+
+    public AdbcException(string message)
+        : base(message)
+    {
+    }
+
+    public AdbcException(string message, AdbcStatusCode statusCode)
+        : base(message)
+    {
+        this.statusCode = statusCode;
+    }
+
+    public AdbcException(string message, AdbcStatusCode statusCode, Exception innerException)
+       : base(message, innerException)
+    {
+    }
+
+    public AdbcException(string message, Exception innerException)
+        : base(message, innerException)
+    {
+    }
+
+    public static AdbcException NotImplemented(string message)
+    {
+        return new AdbcException(message, AdbcStatusCode.NotImplemented);
+    }
+
+    //
+    // Summary:
+    //     For database providers which support it, contains a standard SQL 5-character
+    //     return code indicating the success or failure of the database operation. The
+    //     first 2 characters represent the class of the return code (e.g. error, success),
+    //     while the last 3 characters represent the subclass, allowing detection of error
+    //     scenarios in a database-portable way.
+    //     For database providers which don't support it, or for inapplicable error scenarios,
+    //     contains null.
+    //
+    // Returns:
+    //     A standard SQL 5-character return code, or null.
+    public virtual string SqlState
+    {
+        get { return null; }
+        protected set { throw new NotImplementedException(); }

Review Comment:
   Is the lack of implementation intentional here? Or is this more a TODO?



##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcConnection.cs:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.Ipc;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// Provides methods for query execution, managing prepared statements, using transactions, and so on.
+    /// </summary>
+    public abstract class AdbcConnection : IDisposable

Review Comment:
   What's been done for Go, at least, is to provide a package that implements DbConnection and the other 'standard' DB APIs and wraps any ADBC object underneath, since the motivation is that the 'standard' APIs don't expose Arrow in the first place



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220836644


##########
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:
   I renamed the file - TBD on the other items for now? 



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

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

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


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

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1579017490

   Mid-late June, but we could omit this from the release (it wouldn't be part of it anyways unless/until we wire it into the release scripts)


-- 
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-adbc] CurtHagenlocher commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "CurtHagenlocher (via GitHub)" <gi...@apache.org>.
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


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

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1205823557


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        private static unsafe NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private static NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private static NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private static NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private static NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+        private static NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private static NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private static NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        private static unsafe NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);

Review Comment:
   discussed over chat that these need to remain unsafe because the method it is referencing has pointers in it



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

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

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


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

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1206211338


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.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; }
+
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind()

Review Comment:
   yes and no. it's there to define the Bind method for the AdbcStatement, doesn't force subclasses to implement it, but if they want any functionality then they need to override it.



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

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

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


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

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1207068781


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.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; }
+
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind()
+        {
+            throw new NotImplementedException();
+        }
+
+        /// <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; }
+
+        /// <summary>
+        /// Execute a result set-generating query and get a list of partitions of the result set.
+        /// </summary>
+        /// <returns><see cref="PartitionedResult"/></returns>
+        public virtual PartitionedResult ExecutePartitioned()
+        {
+            throw AdbcException.NotImplemented("Statement does not support executePartitioned");
+        }
+
+        public virtual Schema GetParameterSchema()
+        {
+            throw AdbcException.NotImplemented("Statement does not support GetParameterSchema");
+        }
+
+        public virtual void Prepare()
+        {
+            throw AdbcException.NotImplemented("Statement does not support Prepare");
+        }
+
+        public virtual void Dispose()
+        {
+        }
+
+        /// <summary>
+        /// Gets the .NET type based on the Arrow field metadata
+        /// </summary>
+        /// <param name="f"></param>
+        /// <returns></returns>
+        public virtual Type ConvertArrowType(Field f)

Review Comment:
   If the raw data is of the wrong type, it should be the driver's responsibility to convert it into the right Arrow type, right? The Go Snowflake driver, for instance, casts the Snowflake result data to the right type after inspecting the type metadata, without pushing this onto the user or into the interface.



-- 
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-adbc] lidavidm commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1211920352


##########
csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/flightsql.parquet:
##########


Review Comment:
   Hmm, I guess we're OK checking in binary artifacts for testing for now.
   
   Though question, is there a need for this to be in Parquet or would just storing the data as Arrow directly work?



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1215026675


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;

Review Comment:
   resolved in latest push



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;
+
+        /// <summary>
+        /// The length of each corresponding entry in partitions.
+        /// </summary>
+        public int* partition_lengths;
+
+        /// <summary>
+        /// Opaque implementation-defined state.
+        /// This field is NULLPTR iff the connection is unintialized/freed.
+        /// </summary>
+        public void* private_data;
+
+        /// <summary>
+        /// Release the contained partitions.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        /// </summary>
+        public delegate* unmanaged[Stdcall]<NativeAdbcPartitions*, void> release; 
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcError
+    {
+        /// <summary>
+        /// The error message.
+        /// </summary>
+        public char* message;

Review Comment:
   resolved in latest push



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220822614


##########
csharp/src/Apache.Arrow.Adbc/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
+{
+    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:
   replaced with _descriptor.GetHashCode()



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220820491


##########
csharp/src/Apache.Arrow.Adbc/AdbcConnection.cs:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.Ipc;
+
+namespace Apache.Arrow.Adbc
+{
+    /// <summary>
+    /// Provides methods for query execution, managing prepared statements, using transactions, and so on.
+    /// </summary>
+    public abstract class AdbcConnection : IDisposable
+    {
+        private bool _autoCommit = true;
+        private bool _readOnly = false;
+        private IsolationLevel _isolationLevel = IsolationLevel.Default;

Review Comment:
   I would prefer to leave them as private. That will force implementors to have their own for getting and setting. 



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1205826785


##########
csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj:
##########
@@ -0,0 +1,18 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <Compile Remove="Extensions\MatchCollectionExtensions.netstandard.cs" />
+    <Compile Remove="Interop\NativeMemoryManager.cs" />

Review Comment:
   done



##########
csharp/Directory.Build.targets:
##########
@@ -0,0 +1,29 @@
+<!--
+  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>
+
+  <!-- The following works around https://github.com/dotnet/sourcelink/issues/572  -->

Review Comment:
   done



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1215025569


##########
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 Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Adbc.Tests;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Moq;
+using Newtonsoft.Json;

Review Comment:
   done



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220836901


##########
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:
   done



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220824555


##########
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:
   I only intended for it to be the iteration of this check-in; not necessarily where ADBC is. but, I will defer to @lidavidm  on this one 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-adbc] lidavidm commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1221448184


##########
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:
   Correct, there's not a 'standard' timeout parameter (perhaps we should define one?) but individual drivers often have this



-- 
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-adbc] davidhcoe commented on pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1579238632

   Is this far enough we can commit this iteration?


-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220872281


##########
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:
   re-formatted



-- 
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-adbc] CurtHagenlocher commented on pull request #697: feat(csharp): adding C# functionality

Posted by "CurtHagenlocher (via GitHub)" <gi...@apache.org>.
CurtHagenlocher commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1579013987

   > Admittedly I'm only giving things a cursory look. Even if there's a few parts I'm not sure about, I'd be in favor of merging sooner rather than later and continuing to cycle on things in later PRs.
   
   Is there a target date yet for a 0.5 release? I think there's a bunch of stuff that would need to have cleaned up before then.


-- 
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-adbc] eerhardt commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1221968792


##########
csharp/src/Apache.Arrow.Adbc/Interop/LoadDriver.cs:
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Threading;
+using System.Threading.Tasks;
+using System.Xml.Linq;
+using Apache.Arrow.Adbc;
+using Apache.Arrow.C;
+using Apache.Arrow.Ipc;
+using Apache.Arrow.Types;
+using Microsoft.Win32.SafeHandles;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Interop
+{
+    public delegate byte AdbcDriverInit(int version, ref NativeAdbcDriver driver, ref NativeAdbcError error);
+
+    /// <summary>
+    /// Class for working with loading drivers from files
+    /// </summary>
+    public static class LoadDriver
+    {
+        private const string driverInit = "AdbcDriverInit";
+
+        class NativeDriver
+        {
+            public SafeHandle driverHandle;
+            public NativeAdbcDriver driver;
+        }
+
+        /// <summary>
+        /// Class used for Mac interoperability
+        /// </summary>
+        static class MacInterop
+        {
+            const string libdl = "libdl.dylib";
+
+            [DllImport(libdl)]
+            static extern SafeLibraryHandle dlopen(string fileName, int flags);
+
+            [DllImport(libdl)]
+            static extern IntPtr dlsym(SafeHandle libraryHandle, string symbol);
+
+            [DllImport(libdl)]
+            static extern int dlclose(IntPtr handle);
+
+            sealed class SafeLibraryHandle : SafeHandleZeroOrMinusOneIsInvalid
+            {
+                SafeLibraryHandle() : base(true) { }
+
+                protected override bool ReleaseHandle()
+                {
+                    return dlclose(handle) == 0;
+                }
+            }
+
+            public static NativeDriver GetDriver(string file)
+            {
+                SafeHandle library = dlopen(file, 2); // TODO: find a symbol for 2
+                IntPtr symbol = dlsym(library, "AdbcDriverInit");
+                AdbcDriverInit init = Marshal.GetDelegateForFunctionPointer<AdbcDriverInit>(symbol);
+                NativeAdbcDriver driver = new NativeAdbcDriver();
+                NativeAdbcError error = new NativeAdbcError();
+                byte result = init(1000000, ref driver, ref error);
+                return new NativeDriver { driverHandle = library, driver = driver };
+            }
+        }
+
+        /// <summary>
+        /// Class used for Windows interoperability
+        /// </summary>
+        static class WindowsInterop
+        {
+            const string kernel32 = "kernel32.dll";
+
+            [DllImport(kernel32)]
+            [return: MarshalAs(UnmanagedType.Bool)]
+            static extern bool FreeLibrary(IntPtr libraryHandle);
+
+            [DllImport(kernel32, CharSet = CharSet.Ansi, BestFitMapping = false, ThrowOnUnmappableChar = true)]
+            static extern IntPtr GetProcAddress(SafeHandle libraryHandle, string functionName);
+
+            [DllImport(kernel32, CharSet = CharSet.Unicode, SetLastError = true)]
+            static extern SafeLibraryHandle LoadLibraryEx(string fileName, IntPtr hFile, uint flags);
+
+            sealed class SafeLibraryHandle : SafeHandleZeroOrMinusOneIsInvalid
+            {
+                SafeLibraryHandle() : base(true) { }
+
+                protected override bool ReleaseHandle()
+                {
+                    return FreeLibrary(handle);
+                }
+            }
+
+            public static NativeDriver GetDriver(string file)
+            {
+                SafeHandle library = LoadLibraryEx(file, IntPtr.Zero, 0x1100);
+                IntPtr symbol = GetProcAddress(library, "AdbcDriverInit");
+                AdbcDriverInit init = Marshal.GetDelegateForFunctionPointer<AdbcDriverInit>(symbol);
+                NativeAdbcDriver driver = new NativeAdbcDriver();
+                NativeAdbcError error = new NativeAdbcError();
+                byte result = init(1000000, ref driver, ref error);

Review Comment:
   Are we ignoring the `error` here? And above in the MacInterop.



##########
csharp/src/Apache.Arrow.Adbc/Interop/LoadDriver.cs:
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Threading;
+using System.Threading.Tasks;
+using System.Xml.Linq;
+using Apache.Arrow.Adbc;
+using Apache.Arrow.C;
+using Apache.Arrow.Ipc;
+using Apache.Arrow.Types;
+using Microsoft.Win32.SafeHandles;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Interop
+{
+    public delegate byte AdbcDriverInit(int version, ref NativeAdbcDriver driver, ref NativeAdbcError error);
+
+    /// <summary>
+    /// Class for working with loading drivers from files
+    /// </summary>
+    public static class LoadDriver
+    {
+        private const string driverInit = "AdbcDriverInit";
+
+        class NativeDriver
+        {
+            public SafeHandle driverHandle;
+            public NativeAdbcDriver driver;
+        }
+
+        /// <summary>
+        /// Class used for Mac interoperability
+        /// </summary>
+        static class MacInterop
+        {
+            const string libdl = "libdl.dylib";
+
+            [DllImport(libdl)]
+            static extern SafeLibraryHandle dlopen(string fileName, int flags);
+
+            [DllImport(libdl)]
+            static extern IntPtr dlsym(SafeHandle libraryHandle, string symbol);
+
+            [DllImport(libdl)]
+            static extern int dlclose(IntPtr handle);
+
+            sealed class SafeLibraryHandle : SafeHandleZeroOrMinusOneIsInvalid
+            {
+                SafeLibraryHandle() : base(true) { }
+
+                protected override bool ReleaseHandle()
+                {
+                    return dlclose(handle) == 0;
+                }
+            }
+
+            public static NativeDriver GetDriver(string file)
+            {
+                SafeHandle library = dlopen(file, 2); // TODO: find a symbol for 2
+                IntPtr symbol = dlsym(library, "AdbcDriverInit");
+                AdbcDriverInit init = Marshal.GetDelegateForFunctionPointer<AdbcDriverInit>(symbol);
+                NativeAdbcDriver driver = new NativeAdbcDriver();
+                NativeAdbcError error = new NativeAdbcError();
+                byte result = init(1000000, ref driver, ref error);
+                return new NativeDriver { driverHandle = library, driver = driver };
+            }
+        }
+
+        /// <summary>
+        /// Class used for Windows interoperability
+        /// </summary>
+        static class WindowsInterop
+        {
+            const string kernel32 = "kernel32.dll";
+
+            [DllImport(kernel32)]
+            [return: MarshalAs(UnmanagedType.Bool)]
+            static extern bool FreeLibrary(IntPtr libraryHandle);
+
+            [DllImport(kernel32, CharSet = CharSet.Ansi, BestFitMapping = false, ThrowOnUnmappableChar = true)]
+            static extern IntPtr GetProcAddress(SafeHandle libraryHandle, string functionName);
+
+            [DllImport(kernel32, CharSet = CharSet.Unicode, SetLastError = true)]
+            static extern SafeLibraryHandle LoadLibraryEx(string fileName, IntPtr hFile, uint flags);
+
+            sealed class SafeLibraryHandle : SafeHandleZeroOrMinusOneIsInvalid
+            {
+                SafeLibraryHandle() : base(true) { }
+
+                protected override bool ReleaseHandle()
+                {
+                    return FreeLibrary(handle);
+                }
+            }
+
+            public static NativeDriver GetDriver(string file)
+            {
+                SafeHandle library = LoadLibraryEx(file, IntPtr.Zero, 0x1100);
+                IntPtr symbol = GetProcAddress(library, "AdbcDriverInit");
+                AdbcDriverInit init = Marshal.GetDelegateForFunctionPointer<AdbcDriverInit>(symbol);
+                NativeAdbcDriver driver = new NativeAdbcDriver();
+                NativeAdbcError error = new NativeAdbcError();
+                byte result = init(1000000, ref driver, ref error);
+                return new NativeDriver { /* driverHandle = library, */ driver = driver };
+            }
+        }
+
+        /// <summary>
+        /// Loads an <see cref="AdbcDriver"/> from the file system.
+        /// </summary>
+        /// <param name="file">
+        /// The path to the file.
+        /// </param>
+        public static AdbcDriver Load(string file)
+        {
+            if (file[0] == '/')

Review Comment:
   What's the plan for Linux? Might be good to have a follow-up issue for supporting Linux.



##########
csharp/src/Apache.Arrow.Adbc/Interop.cs:
##########
@@ -0,0 +1,1203 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using Apache.Arrow.Ipc;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionGetObjects> connectionGetObjects = new NativeDelegate<ConnectionGetObjects>(GetConnectionObjects);
+        private unsafe static readonly NativeDelegate<ConnectionGetTableSchema> connectionGetTableSchema = new NativeDelegate<ConnectionGetTableSchema>(GetConnectionTableSchema);
+        private unsafe static readonly NativeDelegate<ConnectionGetTableTypes> connectionGetTableTypes = new NativeDelegate<ConnectionGetTableTypes>(GetConnectionTableTypes);
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionReadPartition> connectionReadPartition = new NativeDelegate<ConnectionReadPartition>(ReadConnectionPartition);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+
+        private unsafe static readonly NativeDelegate<StatementBind> statementBind = new NativeDelegate<StatementBind>(BindStatement);
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            nativeDriver->ConnectionGetObjects = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, int, byte*, byte*, byte*, byte**, byte*, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetObjects.Pointer;
+            nativeDriver->ConnectionGetTableSchema = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, byte*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)connectionGetTableSchema.Pointer;
+            nativeDriver->ConnectionGetTableTypes = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetTableTypes.Pointer;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            nativeDriver->ConnectionReadPartition = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionReadPartition.Pointer;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            nativeDriver->ConnectionRollback = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+
+            nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)statementBind.Pointer;
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError*, AdbcStatusCode>)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeHGlobal((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+#if NETSTANDARD
+                error->message = (byte*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+#else
+            error->message = (byte*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+#endif
+
+            error->sqlstate0 = (byte)0;
+            error->sqlstate1 = (byte)0;
+            error->sqlstate2 = (byte)0;
+            error->sqlstate3 = (byte)0;
+            error->sqlstate4 = (byte)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];

Review Comment:
   Would it make sense to make this a `List<MemoryHandle>` instead, so we don't have to count all the handles first, and then pin them?



##########
dev/release/rat_exclude_files.txt:
##########
@@ -30,3 +30,8 @@ c/vendor/sqlite3/sqlite3.c
 c/vendor/sqlite3/sqlite3.h
 *.Rproj
 *.Rd
+csharp/Apache.Arrow.Adbc.sln
+csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj
+csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj
+csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/Apache.Arrow.Adbc.FlightSql.Tests.csproj
+csharp/test/Apache.Arrow.Adbc.Tests/Apache.Arrow.Adbc.Tests.csproj

Review Comment:
   I didn't know these rat_exclude_files support wildcards. Can we just have:
   
   ```suggestion
   *.sln
   *.csproj
   ```



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220822334


##########
csharp/src/Apache.Arrow.Adbc/AdbcStatement.cs:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.Text.RegularExpressions;
+using System.Threading.Tasks;
+using Apache.Arrow.Ipc;
+
+namespace Apache.Arrow.Adbc
+{
+    /// <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()
+        {
+            
+        }
+
+        /// <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>
+        /// Execute a result set-generating query and get a list of partitions of the result set.
+        /// </summary>
+        /// <returns><see cref="PartitionedResult"/></returns>
+        public virtual PartitionedResult ExecutePartitioned()
+        {
+            throw AdbcException.NotImplemented("Statement does not support ExecutePartitioned");
+        }
+
+        /// <summary>
+        /// Get the schema for bound parameters.
+        /// </summary>
+        /// <returns><see cref="Schema"/></returns>
+        public virtual Schema GetParameterSchema()
+        {
+            throw AdbcException.NotImplemented("Statement does not support GetParameterSchema");
+        }
+
+        /// <summary>
+        ///  Turn this statement into a prepared statement to be
+        ///  executed multiple times.
+        /// </summary>
+        public virtual void Prepare()
+        {
+            throw AdbcException.NotImplemented("Statement does not support Prepare");
+        }
+
+        public virtual void Dispose()
+        {
+        }
+
+        /// <summary>
+        /// Gets a value from the Arrow array at the specified index, using the Field metadata for information.
+        /// </summary>
+        /// <param name="arrowArray">The Arrow array.</param>
+        /// <param name="field">The <see cref="Field"/> from the <see cref="Schema"/> that can be used for metadata inspection.</param>
+        /// <param name="index">The index in the array to get the value from.</param>
+        /// <returns></returns>
+        public abstract object GetValue(IArrowArray arrowArray, Field field, int index);
+
+        /// <summary>
+        /// For decimals, Arrow throws an OverflowException if a value is < decimal.min or > decimal.max
+        /// So parse the numeric value and return it as a string, if possible
+        /// </summary>
+        /// <param name="oex"></param>
+        /// <returns>A string value of the decimal that threw the exception or rethrows the OverflowException.</returns>
+        /// <exception cref="ArgumentNullException"></exception>
+        public virtual string ParseDecimalValueFromOverflowException(OverflowException oex)

Review Comment:
   moved to flightsql implementation



-- 
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-adbc] lidavidm merged pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm merged PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697


-- 
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-adbc] eerhardt commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1205765476


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        private static unsafe NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);

Review Comment:
   ```suggestion
           private static NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
   ```
   
   why is `unsafe` needed here?



##########
csharp/Directory.Build.targets:
##########
@@ -0,0 +1,29 @@
+<!--
+  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>
+
+  <!-- The following works around https://github.com/dotnet/sourcelink/issues/572  -->

Review Comment:
   Let's leave this out until we need it. This issue should be fixed in recent versions of the SDK. Can you delete this file?



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop

Review Comment:
   Does this _need_ to be public?



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        private static unsafe NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private static NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private static NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private static NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private static NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+        private static NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private static NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private static NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        private static unsafe NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private static NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private static NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private static NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            (*nativeDriver).private_data = GCHandle.ToIntPtr(handle);
+            (*nativeDriver).release = releaseDriver;
+            (*nativeDriver).DatabaseNew = stub.newDatabase;
+            (*nativeDriver).DatabaseInit = databaseInit;
+            (*nativeDriver).DatabaseRelease = databaseRelease;
+            (*nativeDriver).DatabaseSetOption = databaseSetOption;
+            (*nativeDriver).ConnectionNew = stub.newConnection;
+            (*nativeDriver).ConnectionInit = connectionInit;
+            (*nativeDriver).ConnectionRelease = connectionRelease;
+            (*nativeDriver).ConnectionSetOption = connectionSetOption;
+            (*nativeDriver).StatementNew = statementNew;
+            (*nativeDriver).StatementSetSqlQuery = statementSetSqlQuery;
+            (*nativeDriver).StatementExecuteQuery = statementExecuteQuery;
+            (*nativeDriver).StatementRelease = statementRelease;
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && (*error).message != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((*error).message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+#if NETSTANDARD
+            error->message = MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+#else
+            error->message = Marshal.StringToCoTaskMemUTF8(exception.Message);
+#endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)(IntPtr)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private static AdbcStatusCode ReleaseDriver(ref NativeAdbcDriver nativeDriver, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeDriver.private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver.private_data = IntPtr.Zero;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(ref NativeAdbcDatabase nativeDatabase, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(ref NativeAdbcDatabase nativeDatabase, ref NativeAdbcError error)
+        {
+            if ((IntPtr)nativeDatabase.private_data == IntPtr.Zero)

Review Comment:
   ```suggestion
               if (nativeDatabase.private_data == null)
   ```



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        private static unsafe NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private static NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private static NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private static NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private static NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+        private static NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private static NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private static NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        private static unsafe NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private static NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private static NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private static NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            (*nativeDriver).private_data = GCHandle.ToIntPtr(handle);
+            (*nativeDriver).release = releaseDriver;
+            (*nativeDriver).DatabaseNew = stub.newDatabase;
+            (*nativeDriver).DatabaseInit = databaseInit;
+            (*nativeDriver).DatabaseRelease = databaseRelease;
+            (*nativeDriver).DatabaseSetOption = databaseSetOption;
+            (*nativeDriver).ConnectionNew = stub.newConnection;
+            (*nativeDriver).ConnectionInit = connectionInit;
+            (*nativeDriver).ConnectionRelease = connectionRelease;
+            (*nativeDriver).ConnectionSetOption = connectionSetOption;
+            (*nativeDriver).StatementNew = statementNew;
+            (*nativeDriver).StatementSetSqlQuery = statementSetSqlQuery;
+            (*nativeDriver).StatementExecuteQuery = statementExecuteQuery;
+            (*nativeDriver).StatementRelease = statementRelease;
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && (*error).message != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((*error).message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+#if NETSTANDARD
+            error->message = MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+#else
+            error->message = Marshal.StringToCoTaskMemUTF8(exception.Message);
+#endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)(IntPtr)releaseError.Pointer;

Review Comment:
   ```suggestion
               error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
   ```



##########
csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj:
##########
@@ -0,0 +1,18 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <Compile Remove="Extensions\MatchCollectionExtensions.netstandard.cs" />

Review Comment:
   Does this need a `Condition=TFM == netstandard` on it?



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        private static unsafe NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private static NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private static NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);

Review Comment:
   ```suggestion
           private static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
   ```
   
   These can all be `readonly`.



##########
csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj:
##########
@@ -0,0 +1,18 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <Compile Remove="Extensions\MatchCollectionExtensions.netstandard.cs" />
+    <Compile Remove="Interop\NativeMemoryManager.cs" />

Review Comment:
   ```suggestion
   
   ```
   
   This can be removed.



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        private static unsafe NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private static NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private static NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private static NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private static NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+        private static NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private static NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private static NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        private static unsafe NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);

Review Comment:
   ```suggestion
           private static NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
   ```



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        private static unsafe NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private static NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private static NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private static NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private static NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+        private static NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private static NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private static NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        private static unsafe NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private static NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private static NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private static NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            (*nativeDriver).private_data = GCHandle.ToIntPtr(handle);
+            (*nativeDriver).release = releaseDriver;
+            (*nativeDriver).DatabaseNew = stub.newDatabase;
+            (*nativeDriver).DatabaseInit = databaseInit;
+            (*nativeDriver).DatabaseRelease = databaseRelease;
+            (*nativeDriver).DatabaseSetOption = databaseSetOption;
+            (*nativeDriver).ConnectionNew = stub.newConnection;
+            (*nativeDriver).ConnectionInit = connectionInit;
+            (*nativeDriver).ConnectionRelease = connectionRelease;
+            (*nativeDriver).ConnectionSetOption = connectionSetOption;
+            (*nativeDriver).StatementNew = statementNew;
+            (*nativeDriver).StatementSetSqlQuery = statementSetSqlQuery;
+            (*nativeDriver).StatementExecuteQuery = statementExecuteQuery;
+            (*nativeDriver).StatementRelease = statementRelease;
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && (*error).message != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((*error).message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+#if NETSTANDARD
+            error->message = MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+#else
+            error->message = Marshal.StringToCoTaskMemUTF8(exception.Message);
+#endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)(IntPtr)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private static AdbcStatusCode ReleaseDriver(ref NativeAdbcDriver nativeDriver, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeDriver.private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver.private_data = IntPtr.Zero;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(ref NativeAdbcDatabase nativeDatabase, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(ref NativeAdbcDatabase nativeDatabase, ref NativeAdbcError error)
+        {
+            if ((IntPtr)nativeDatabase.private_data == IntPtr.Zero)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase.private_data = null;//IntPtr.Zero;

Review Comment:
   ```suggestion
               nativeDatabase.private_data = null;
   ```



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1206209233


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,595 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        static NativePointer<DriverRelease> releaseDriver = new NativePointer<DriverRelease>(ReleaseDriver);
+
+        static NativePointer<DatabaseFn> databaseInit = new NativePointer<DatabaseFn>(InitDatabase);
+        static NativePointer<DatabaseFn> databaseRelease = new NativePointer<DatabaseFn>(ReleaseDatabase);
+        static NativePointer<DatabaseSetOption> databaseSetOption = new NativePointer<DatabaseSetOption>(SetDatabaseOption);
+        static NativePointer<ConnectionInit> connectionInit = new NativePointer<ConnectionInit>(InitConnection);
+        static NativePointer<ConnectionFn> connectionRelease = new NativePointer<ConnectionFn>(ReleaseConnection);
+        static NativePointer<ConnectionSetOption> connectionSetOption = new NativePointer<ConnectionSetOption>(SetConnectionOption);
+        static unsafe NativePointer<StatementExecuteQuery> statementExecuteQuery = new NativePointer<StatementExecuteQuery>(ExecuteStatementQuery);
+        static NativePointer<StatementNew> statementNew = new NativePointer<StatementNew>(NewStatement);
+        static NativePointer<StatementFn> statementRelease = new NativePointer<StatementFn>(ReleaseStatement);
+        static NativePointer<StatementSetSqlQuery> statementSetSqlQuery = new NativePointer<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        unsafe static IntPtr errorRelease = new NativePointer<ErrorRelease>(ReleaseError);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            (*nativeDriver).private_data = GCHandle.ToIntPtr(handle);
+            (*nativeDriver).release = releaseDriver;
+            (*nativeDriver).DatabaseNew = stub.newDatabase;
+            (*nativeDriver).DatabaseInit = databaseInit;
+            (*nativeDriver).DatabaseRelease = databaseRelease;
+            (*nativeDriver).DatabaseSetOption = databaseSetOption;
+            (*nativeDriver).ConnectionNew = stub.newConnection;
+            (*nativeDriver).ConnectionInit = connectionInit;
+            (*nativeDriver).ConnectionRelease = connectionRelease;
+            (*nativeDriver).ConnectionSetOption = connectionSetOption;
+            (*nativeDriver).StatementNew = statementNew;
+            (*nativeDriver).StatementSetSqlQuery = statementSetSqlQuery;
+            (*nativeDriver).StatementExecuteQuery = statementExecuteQuery;
+            (*nativeDriver).StatementRelease = statementRelease;
+            return 0;
+        }
+
+        unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && (*error).message != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((*error).message);
+            }
+        }
+
+        unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+#if NETSTANDARD
+            (*error).message = MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+#else
+            (*error).message = Marshal.StringToCoTaskMemUTF8(exception.Message);
+#endif
+
+            (*error).sqlstate0 = (char)0;
+            (*error).sqlstate1 = (char)0;
+            (*error).sqlstate2 = (char)0;
+            (*error).sqlstate3 = (char)0;
+            (*error).sqlstate4 = (char)0;
+            (*error).vendor_code = 0;
+            (*error).vendor_code = 0;
+            (*error).release = errorRelease;
+
+            return AdbcStatusCode.UnknownError;
+        }
+
+        sealed class PinnedArray : IDisposable
+        {
+            IArrowArray array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                this.array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (array != null)
+                {
+                    array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        static AdbcStatusCode ReleaseDriver(ref NativeAdbcDriver nativeDriver, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeDriver.private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver.private_data = IntPtr.Zero;
+            return 0;
+        }
+
+        static AdbcStatusCode InitDatabase(ref NativeAdbcDatabase nativeDatabase, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref error);
+        }
+
+        static AdbcStatusCode ReleaseDatabase(ref NativeAdbcDatabase nativeDatabase, ref NativeAdbcError error)
+        {
+            if (nativeDatabase.private_data == IntPtr.Zero)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr(nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase.private_data = IntPtr.Zero;
+            return AdbcStatusCode.Success;
+        }
+
+        static AdbcStatusCode SetDatabaseOption(ref NativeAdbcDatabase nativeDatabase, IntPtr name, IntPtr value, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.SetOption(name, value, ref error);
+        }
+
+        static AdbcStatusCode InitConnection(ref NativeAdbcConnection nativeConnection, ref NativeAdbcDatabase database, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeConnection.private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref database, ref error);
+        }
+
+        static AdbcStatusCode ReleaseConnection(ref NativeAdbcConnection nativeConnection, ref NativeAdbcError error)
+        {
+            if (nativeConnection.private_data == IntPtr.Zero)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr(nativeConnection.private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection.private_data = IntPtr.Zero;
+            return AdbcStatusCode.Success;
+        }
+
+        static AdbcStatusCode SetConnectionOption(ref NativeAdbcConnection nativeConnection, IntPtr name, IntPtr value, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeConnection.private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref error);
+        }
+
+        static AdbcStatusCode SetStatementSqlQuery(ref NativeAdbcStatement nativeStatement, IntPtr text, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeStatement.private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+#if NETSTANDARD
+            stub.SqlQuery = MarshalExtensions.PtrToStringUTF8(text);
+#else
+            stub.SqlQuery = Marshal.PtrToStringUTF8(text);
+#endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        static unsafe AdbcStatusCode ExecuteStatementQuery(ref NativeAdbcStatement nativeStatement, CArrowArrayStream* stream, long* rows, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeStatement.private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        static AdbcStatusCode NewStatement(ref NativeAdbcConnection nativeConnection, ref NativeAdbcStatement nativeStatement, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeConnection.private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref nativeStatement, ref error);
+        }
+
+        static AdbcStatusCode ReleaseStatement(ref NativeAdbcStatement nativeStatement, ref NativeAdbcError error)
+        {
+            if (nativeStatement.private_data == IntPtr.Zero)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr(nativeStatement.private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement.private_data = IntPtr.Zero;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcDatabase
+    {
+        public IntPtr private_data;
+        public IntPtr private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcConnection
+    {
+        public IntPtr private_data;
+        public IntPtr private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcStatement
+    {
+        public IntPtr private_data;
+        public IntPtr private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcPartitions
+    {
+        /// \brief The number of partitions.
+        public IntPtr num_partitions;
+
+        /// \brief The partitions of the result set, where each entry (up to
+        ///   num_partitions entries) is an opaque identifier that can be
+        ///   passed to AdbcConnectionReadPartition.
+        public IntPtr partitions;
+
+        /// \brief The length of each corresponding entry in partitions.
+        public IntPtr partition_lengths;
+
+        /// \brief Opaque implementation-defined state.
+        /// This field is NULLPTR iff the connection is unintialized/freed.
+        public IntPtr private_data;
+
+        /// \brief Release the contained partitions.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        public IntPtr release; // void (*release)(struct AdbcPartitions* partitions);
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcError
+    {
+        /// \brief The error message.
+        public IntPtr message;
+
+        /// \brief A vendor-specific error code, if applicable.
+        public int vendor_code;
+
+        /// \brief A SQLSTATE error code, if provided, as defined by the
+        ///   SQL:2003 standard.  If not set, it should be set to
+        ///   "\0\0\0\0\0".
+        public char sqlstate0;
+        public char sqlstate1;
+        public char sqlstate2;
+        public char sqlstate3;
+        public char sqlstate4;
+
+        /// \brief Release the contained error.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        public IntPtr release; // void (*release)(struct AdbcError* error);
+    };
+
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcDriver
+    {
+        public IntPtr private_data;
+        public IntPtr private_manager;

Review Comment:
   working through these



-- 
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-adbc] eerhardt commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1201000245


##########
csharp/src/Apache.Arrow.Adbc/AssemblyInfo.cs:
##########
@@ -0,0 +1,2 @@
+using System.Reflection;

Review Comment:
   This can be deleted.



##########
csharp/src/Apache.Arrow.Adbc/Interop/NativePointer.cs:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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
+{
+    struct NativePointer<T>

Review Comment:
   ```suggestion
       internal readonly struct NativeDelegate<T>
   ```
   
   To follow https://github.com/apache/arrow/blob/main/csharp/src/Apache.Arrow/C/NativeDelegate.cs



##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,595 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        static NativePointer<DriverRelease> releaseDriver = new NativePointer<DriverRelease>(ReleaseDriver);
+
+        static NativePointer<DatabaseFn> databaseInit = new NativePointer<DatabaseFn>(InitDatabase);
+        static NativePointer<DatabaseFn> databaseRelease = new NativePointer<DatabaseFn>(ReleaseDatabase);
+        static NativePointer<DatabaseSetOption> databaseSetOption = new NativePointer<DatabaseSetOption>(SetDatabaseOption);
+        static NativePointer<ConnectionInit> connectionInit = new NativePointer<ConnectionInit>(InitConnection);
+        static NativePointer<ConnectionFn> connectionRelease = new NativePointer<ConnectionFn>(ReleaseConnection);
+        static NativePointer<ConnectionSetOption> connectionSetOption = new NativePointer<ConnectionSetOption>(SetConnectionOption);
+        static unsafe NativePointer<StatementExecuteQuery> statementExecuteQuery = new NativePointer<StatementExecuteQuery>(ExecuteStatementQuery);
+        static NativePointer<StatementNew> statementNew = new NativePointer<StatementNew>(NewStatement);
+        static NativePointer<StatementFn> statementRelease = new NativePointer<StatementFn>(ReleaseStatement);
+        static NativePointer<StatementSetSqlQuery> statementSetSqlQuery = new NativePointer<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        unsafe static IntPtr errorRelease = new NativePointer<ErrorRelease>(ReleaseError);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            (*nativeDriver).private_data = GCHandle.ToIntPtr(handle);
+            (*nativeDriver).release = releaseDriver;
+            (*nativeDriver).DatabaseNew = stub.newDatabase;
+            (*nativeDriver).DatabaseInit = databaseInit;
+            (*nativeDriver).DatabaseRelease = databaseRelease;
+            (*nativeDriver).DatabaseSetOption = databaseSetOption;
+            (*nativeDriver).ConnectionNew = stub.newConnection;
+            (*nativeDriver).ConnectionInit = connectionInit;
+            (*nativeDriver).ConnectionRelease = connectionRelease;
+            (*nativeDriver).ConnectionSetOption = connectionSetOption;
+            (*nativeDriver).StatementNew = statementNew;
+            (*nativeDriver).StatementSetSqlQuery = statementSetSqlQuery;
+            (*nativeDriver).StatementExecuteQuery = statementExecuteQuery;
+            (*nativeDriver).StatementRelease = statementRelease;
+            return 0;
+        }
+
+        unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && (*error).message != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((*error).message);
+            }
+        }
+
+        unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+#if NETSTANDARD
+            (*error).message = MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+#else
+            (*error).message = Marshal.StringToCoTaskMemUTF8(exception.Message);
+#endif
+
+            (*error).sqlstate0 = (char)0;
+            (*error).sqlstate1 = (char)0;
+            (*error).sqlstate2 = (char)0;
+            (*error).sqlstate3 = (char)0;
+            (*error).sqlstate4 = (char)0;
+            (*error).vendor_code = 0;
+            (*error).vendor_code = 0;
+            (*error).release = errorRelease;
+
+            return AdbcStatusCode.UnknownError;
+        }
+
+        sealed class PinnedArray : IDisposable
+        {
+            IArrowArray array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                this.array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (array != null)
+                {
+                    array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        static AdbcStatusCode ReleaseDriver(ref NativeAdbcDriver nativeDriver, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeDriver.private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver.private_data = IntPtr.Zero;
+            return 0;
+        }
+
+        static AdbcStatusCode InitDatabase(ref NativeAdbcDatabase nativeDatabase, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref error);
+        }
+
+        static AdbcStatusCode ReleaseDatabase(ref NativeAdbcDatabase nativeDatabase, ref NativeAdbcError error)
+        {
+            if (nativeDatabase.private_data == IntPtr.Zero)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr(nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase.private_data = IntPtr.Zero;
+            return AdbcStatusCode.Success;
+        }
+
+        static AdbcStatusCode SetDatabaseOption(ref NativeAdbcDatabase nativeDatabase, IntPtr name, IntPtr value, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeDatabase.private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.SetOption(name, value, ref error);
+        }
+
+        static AdbcStatusCode InitConnection(ref NativeAdbcConnection nativeConnection, ref NativeAdbcDatabase database, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeConnection.private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref database, ref error);
+        }
+
+        static AdbcStatusCode ReleaseConnection(ref NativeAdbcConnection nativeConnection, ref NativeAdbcError error)
+        {
+            if (nativeConnection.private_data == IntPtr.Zero)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr(nativeConnection.private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection.private_data = IntPtr.Zero;
+            return AdbcStatusCode.Success;
+        }
+
+        static AdbcStatusCode SetConnectionOption(ref NativeAdbcConnection nativeConnection, IntPtr name, IntPtr value, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeConnection.private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref error);
+        }
+
+        static AdbcStatusCode SetStatementSqlQuery(ref NativeAdbcStatement nativeStatement, IntPtr text, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeStatement.private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+#if NETSTANDARD
+            stub.SqlQuery = MarshalExtensions.PtrToStringUTF8(text);
+#else
+            stub.SqlQuery = Marshal.PtrToStringUTF8(text);
+#endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        static unsafe AdbcStatusCode ExecuteStatementQuery(ref NativeAdbcStatement nativeStatement, CArrowArrayStream* stream, long* rows, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeStatement.private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        static AdbcStatusCode NewStatement(ref NativeAdbcConnection nativeConnection, ref NativeAdbcStatement nativeStatement, ref NativeAdbcError error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(nativeConnection.private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref nativeStatement, ref error);
+        }
+
+        static AdbcStatusCode ReleaseStatement(ref NativeAdbcStatement nativeStatement, ref NativeAdbcError error)
+        {
+            if (nativeStatement.private_data == IntPtr.Zero)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr(nativeStatement.private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement.private_data = IntPtr.Zero;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcDatabase
+    {
+        public IntPtr private_data;
+        public IntPtr private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcConnection
+    {
+        public IntPtr private_data;
+        public IntPtr private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcStatement
+    {
+        public IntPtr private_data;
+        public IntPtr private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcPartitions
+    {
+        /// \brief The number of partitions.
+        public IntPtr num_partitions;
+
+        /// \brief The partitions of the result set, where each entry (up to
+        ///   num_partitions entries) is an opaque identifier that can be
+        ///   passed to AdbcConnectionReadPartition.
+        public IntPtr partitions;
+
+        /// \brief The length of each corresponding entry in partitions.
+        public IntPtr partition_lengths;
+
+        /// \brief Opaque implementation-defined state.
+        /// This field is NULLPTR iff the connection is unintialized/freed.
+        public IntPtr private_data;
+
+        /// \brief Release the contained partitions.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        public IntPtr release; // void (*release)(struct AdbcPartitions* partitions);
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcError
+    {
+        /// \brief The error message.
+        public IntPtr message;
+
+        /// \brief A vendor-specific error code, if applicable.
+        public int vendor_code;
+
+        /// \brief A SQLSTATE error code, if provided, as defined by the
+        ///   SQL:2003 standard.  If not set, it should be set to
+        ///   "\0\0\0\0\0".
+        public char sqlstate0;
+        public char sqlstate1;
+        public char sqlstate2;
+        public char sqlstate3;
+        public char sqlstate4;
+
+        /// \brief Release the contained error.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        public IntPtr release; // void (*release)(struct AdbcError* error);
+    };
+
+
+    [StructLayout(LayoutKind.Sequential)]
+    public struct NativeAdbcDriver
+    {
+        public IntPtr private_data;
+        public IntPtr private_manager;

Review Comment:
   Since this is `public` API - let's follow the same pattern that Apache.Arrow C API uses. Let's not use `IntPtr` in public API for things that are pointers. See https://github.com/apache/arrow/pull/34133#discussion_r1124941545 for more information/reasoning.



##########
csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj:
##########
@@ -0,0 +1,13 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <ProjectReference Include="..\arrow\csharp\src\Apache.Arrow\Apache.Arrow.csproj" />

Review Comment:
   I assume this only works locally. Can we reference a nightly build of Apache.Arrow?



##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcConnection.cs:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.Ipc;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// Provides methods for query execution, managing prepared statements, using transactions, and so on.
+    /// </summary>
+    public abstract class AdbcConnection : IDisposable

Review Comment:
   I'm totally new to Adbc, so forgive the naïve question.
   
   Can this (and other classes) inherit from `DbConnection`/`DbCommand`/etc so things that work with ADO.NET "just work" with Apache.Arrow.Adbc?



##########
csharp/src/Apache.Arrow.Adbc/Interop/NativePointer.cs:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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
+{
+    struct NativePointer<T>
+    {
+        readonly T managedDelegate; // For lifetime management
+        readonly IntPtr nativePointer;
+
+        public NativePointer(T managedDelegate)
+        {
+            this.managedDelegate = managedDelegate;

Review Comment:
   Can this code use the same style as apache/arrow/csharp uses? Basically this:
   
   https://github.com/dotnet/runtime/blob/main/docs/coding-guidelines/coding-style.md



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1211006098


##########
csharp/test/Apache.Arrow.Adbc.Tests/DriverConnectionTests.cs:
##########
@@ -0,0 +1,35 @@
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+
+namespace Apache.Arrow.Adbc.Tests
+{
+    /// <summary>
+    /// Abstract class for the ADBC connection tests.
+    /// </summary>
+    public abstract class DriverConnectionTests

Review Comment:
   moved to a model where the base Adbc.Tests project does the validation, but the respective drivers are responsible for passing in the values to be validated



-- 
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-adbc] lidavidm commented on pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1557836265

   Cool! 
   
   It's been >15 years since I last touched C♯, so this will take me a while to review (or I will have to see if Weston or someone else can help).


-- 
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-adbc] github-actions[bot] commented on pull request #697: adding C# functionality

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1557710856

   :warning: Please follow the [Conventional Commits format in CONTRIBUTING.md](https://github.com/apache/arrow-adbc/blob/main/CONTRIBUTING.md) for PR titles.


-- 
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-adbc] lidavidm commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1221448576


##########
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:
   We can update it later (but it would be fine to put 0.5.0 here)



-- 
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-adbc] eerhardt commented on pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1581483309

   @lidavidm - It looks like I have merge permissions here since I'm a committer on apache/arrow. But I'm unsure on the process. Can I just click the green button in GitHub? In apache/arrow there is a python script I run: `dev/merge_arrow_pr.py`, but I'm not seeing it here.
   
   If you want to merge this PR, please go ahead.


-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220824034


##########
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:
   I copied over what was in the Arrow one, but changed to 2022-2023. unless @lidavidm  has an opinion on it.



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

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

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


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

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220870958


##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache - Backup.Arrow.Adbc.FlightSql.csproj:
##########


Review Comment:
   removed



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220836416


##########
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:
   fixed



-- 
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-adbc] lidavidm commented on pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1580962799

   There's still license header issues; if need be you can add paths to https://github.com/apache/arrow-adbc/blob/main/dev/release/rat_exclude_files.txt


-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1215028041


##########
csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj:
##########
@@ -0,0 +1,13 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <Version>0.1.0</Version>

Review Comment:
   resolved in latest push



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1218458661


##########
csharp/src/Apache.Arrow.Adbc/Extensions/MarshalExtensions.netstandard.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.
+ */
+
+#if NETSTANDARD
+using System;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Text;
+
+namespace Apache.Arrow.Adbc.Extensions
+{
+    public static class MarshalExtensions
+    {
+        public static unsafe string PtrToStringUTF8(IntPtr intPtr)
+        {
+            if (intPtr == null) 
+            {
+                return null;
+            }
+
+            unsafe
+            {
+                byte* source = (byte*)intPtr;
+                int nbBytes =  source[0];

Review Comment:
   added a different approach



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1213262004


##########
csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/flightsql.parquet:
##########


Review Comment:
   good idea. removed Parquet in favor of arrow files



-- 
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-adbc] lidavidm commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1202175572


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.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; }
+
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind()
+        {
+            throw new NotImplementedException();
+        }
+
+        /// <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()

Review Comment:
   Is it normal to have sync and async variants of methods in the same class? Or would they be separate interfaces?



##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.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; }
+
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind()
+        {
+            throw new NotImplementedException();
+        }
+
+        /// <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; }
+
+        /// <summary>
+        /// Execute a result set-generating query and get a list of partitions of the result set.
+        /// </summary>
+        /// <returns><see cref="PartitionedResult"/></returns>
+        public virtual PartitionedResult ExecutePartitioned()
+        {
+            throw AdbcException.NotImplemented("Statement does not support executePartitioned");
+        }
+
+        public virtual Schema GetParameterSchema()
+        {
+            throw AdbcException.NotImplemented("Statement does not support GetParameterSchema");
+        }
+
+        public virtual void Prepare()
+        {
+            throw AdbcException.NotImplemented("Statement does not support Prepare");
+        }
+
+        public virtual void Dispose()
+        {
+        }
+
+        /// <summary>
+        /// Gets the .NET type based on the Arrow field metadata
+        /// </summary>
+        /// <param name="f"></param>
+        /// <returns></returns>
+        public virtual Type ConvertArrowType(Field f)

Review Comment:
   It seems the below helpers should go somewhere else? Possibly even upstream into the Arrow library itself?



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlParameters.cs:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// Parameters used for connecting to Flight SQL data sources.
+    /// </summary>
+    public class FlightSqlParameters
+    {
+        public const string ServerAddress = "FLIGHT_SQL_SERVER_ADRESS";

Review Comment:
   ```suggestion
           public const string ServerAddress = "FLIGHT_SQL_SERVER_ADDRESS";
   
   ```



##########
csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/DriverConnectionTests.cs:
##########
@@ -0,0 +1,81 @@
+using System.Collections.Generic;
+using System.IO;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Adbc.FlightSql;
+using Apache.Arrow.Adbc.FlightSql.Tests;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Newtonsoft.Json;
+
+namespace Apache.Arrow.Adbc.Tests
+{
+    [TestClass]
+    public class FlightSqlDriverConnectionTests : DriverConnectionTests
+    {
+        [TestMethod]
+        public override 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();
+
+            long count = 0;
+
+            while (true)
+            {
+                var nextBatch = queryResult.Stream.ReadNextRecordBatchAsync().Result;
+                if (nextBatch == null) { break; }
+                count += nextBatch.Length;
+            }
+
+            Assert.AreEqual(flightSqlTestConfiguration.ExpectedResultsCount, count);
+        }
+    
+        public override void CanDriverUpdate()
+        {
+            throw new System.NotImplementedException();
+        }
+
+        public override void CanReadSchema()
+        {
+            throw new System.NotImplementedException();
+        }
+
+        public override void VerifyBadQueryGeneratesError()
+        {
+            throw new System.NotImplementedException();
+        }
+
+        public override void VerifyTypesAndValues()
+        {
+            throw new System.NotImplementedException();
+        }
+
+        private FlightSqlTestConfiguration GetFlightSqlTestConfiguration()
+        {
+            string json = File.ReadAllText("flightsqlconfig.pass");

Review Comment:
   I see this is explicitly excluded; it seems we need it to actually run tests though?



##########
csharp/src/Apache.Arrow.Adbc/Extensions/MarshalExtensions.netstandard.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.
+ */
+
+#if NETSTANDARD
+using System;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Text;
+
+namespace Apache.Arrow.Adbc.Extensions 
+{
+    public static class MarshalExtensions
+    {
+        public static unsafe string PtrToStringUTF8(IntPtr intPtr)
+        {
+            if (intPtr == null) // IsNullOrWin32Atom(intPtr))

Review Comment:
   What's the intent of the commented code?



##########
csharp/src/Apache.Arrow.Adbc/Extensions/MarshalExtensions.netstandard.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.
+ */
+
+#if NETSTANDARD
+using System;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Text;
+
+namespace Apache.Arrow.Adbc.Extensions 
+{
+    public static class MarshalExtensions
+    {
+        public static unsafe string PtrToStringUTF8(IntPtr intPtr)
+        {
+            if (intPtr == null) // IsNullOrWin32Atom(intPtr))

Review Comment:
   Ah, are these methods backports from newer .NET versions?



##########
csharp/src/Apache.Arrow.Adbc/Extensions/MatchCollectionExtensions.netstandard.cs:
##########
@@ -0,0 +1,23 @@
+#if NETSTANDARD
+using System.Collections.Generic;
+using System.Linq;
+using System.Text.RegularExpressions;
+
+namespace System.Text.RegularExpressions
+{
+    public static class MatchCollectionExtensions
+    {
+        public static IEnumerable<Match> Where(this MatchCollection matchCollection, Func<Match,bool> predicate)

Review Comment:
   Is it usual for a library to add its own extension methods to standard library types?



##########
csharp/test/Apache.Arrow.Adbc.Tests/DriverConnectionTests.cs:
##########
@@ -0,0 +1,35 @@
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+
+namespace Apache.Arrow.Adbc.Tests
+{
+    /// <summary>
+    /// Abstract class for the ADBC connection tests.
+    /// </summary>
+    public abstract class DriverConnectionTests

Review Comment:
   The Java/Go/C++ implementations have a generic test suite written in terms of the public API only along with hooks for individual drivers to inject connection options, etc. Without that I'm not sure there's value in purely a common interface for tests



##########
csharp/src/Apache.Arrow.Adbc/Extensions/MatchCollectionExtensions.netstandard.cs:
##########
@@ -0,0 +1,23 @@
+#if NETSTANDARD
+using System.Collections.Generic;
+using System.Linq;
+using System.Text.RegularExpressions;
+
+namespace System.Text.RegularExpressions
+{
+    public static class MatchCollectionExtensions
+    {
+        public static IEnumerable<Match> Where(this MatchCollection matchCollection, Func<Match,bool> predicate)
+        {
+            List<Match> matches = new List<Match>();
+
+            foreach (Match match in matchCollection) 
+            { 
+                matches.Add(match);
+            }
+            
+            return matches.Where(predicate);

Review Comment:
   Could you avoid Where and just apply the predicate yourself in the loop?



##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.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; }
+
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind()

Review Comment:
   This is missing the actual data to bind?



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlStatement.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.Threading.Tasks;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Flight;
+using Grpc.Core;
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// A Flight SQL implementation of <see cref="AdbcStatement"/>.
+    /// </summary>
+    public class FlightSqlStatement : AdbcStatement
+    {
+        private FlightSqlConnection flightSqlConnection;
+        
+        public FlightSqlStatement(FlightSqlConnection flightSqlConnection)
+        {
+            this.flightSqlConnection = flightSqlConnection;
+        }
+
+        public override async ValueTask<QueryResult> ExecuteQueryAsync()
+        {
+            FlightInfo info = await GetInfo(this.SqlQuery, this.flightSqlConnection.Metadata);
+
+            return new QueryResult(info.TotalRecords, new FlightSqlResult(this.flightSqlConnection, info));
+        }
+
+        public override QueryResult ExecuteQuery()
+        {
+            return ExecuteQueryAsync().Result;
+        }
+
+        public override UpdateResult ExecuteUpdate()
+        {
+            throw new NotImplementedException();
+        }
+
+        public async ValueTask<FlightInfo> GetInfo(string query, Metadata headers)
+        {
+            FlightDescriptor commandDescripter = FlightDescriptor.CreateCommandDescriptor(query);
+
+            return await this.flightSqlConnection.FlightClient.GetInfo(commandDescripter, headers).ResponseAsync;
+        }
+
+        /// <summary>
+        /// Gets a value from the Arrow array at the specified index using the Arrow field for metadata.
+        /// </summary>
+        /// <param name="arrowArray"></param>
+        /// <param name="field"></param>
+        /// <param name="index"></param>
+        /// <returns></returns>
+        public override object GetValue(IArrowArray arrowArray, Field field, int index)

Review Comment:
   Same here - this seems like it belongs somewhere else?



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1221590162


##########
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:
   done



-- 
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-adbc] lidavidm commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1221448981


##########
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:
   Hmm we should update the Arrow one but 2022-2023 is fine here



-- 
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-adbc] lidavidm commented on pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1581486928

   If possible - can we create GitHub issues for followups? (I suppose you may have your own work queue but it'd be nice to make at least some of that public)


-- 
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-adbc] lidavidm commented on pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1581486099

   Yup, we just use the button here.


-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1218462987


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);

Review Comment:
   switched to FreeHGlobal



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220839402


##########
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 ended up removing this, but we can talk about whether it's relevant.  The spec doesn't call for a timeout value, but I know of at least one driver that benefits from the value.



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220839590


##########
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:
   done



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220836189


##########
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:
   I wasn't able to do that because I cant add an actual extension to Marshal since it's static



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220838734


##########
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:
   done



##########
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:
   done



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1213274512


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.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; }
+
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind()
+        {
+            throw new NotImplementedException();
+        }
+
+        /// <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; }
+
+        /// <summary>
+        /// Execute a result set-generating query and get a list of partitions of the result set.
+        /// </summary>
+        /// <returns><see cref="PartitionedResult"/></returns>
+        public virtual PartitionedResult ExecutePartitioned()
+        {
+            throw AdbcException.NotImplemented("Statement does not support executePartitioned");
+        }
+
+        public virtual Schema GetParameterSchema()
+        {
+            throw AdbcException.NotImplemented("Statement does not support GetParameterSchema");
+        }
+
+        public virtual void Prepare()
+        {
+            throw AdbcException.NotImplemented("Statement does not support Prepare");
+        }
+
+        public virtual void Dispose()
+        {
+        }
+
+        /// <summary>
+        /// Gets the .NET type based on the Arrow field metadata
+        /// </summary>
+        /// <param name="f"></param>
+        /// <returns></returns>
+        public virtual Type ConvertArrowType(Field f)

Review Comment:
   right -- similar to https://github.com/apache/arrow-adbc/blob/main/go/adbc/driver/snowflake/statement.go



-- 
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-adbc] eerhardt commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1213675613


##########
csharp/src/Apache.Arrow.Adbc/Interop/LoadDriver.cs:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Threading;
+using System.Threading.Tasks;
+using System.Xml.Linq;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.C;
+using Apache.Arrow.Ipc;
+using Apache.Arrow.Types;
+using Microsoft.Win32.SafeHandles;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Interop
+{
+    public delegate byte AdbcDriverInit(int version, ref NativeAdbcDriver driver, ref NativeAdbcError error);
+
+    /// <summary>
+    /// Class for working with loading drivers from files
+    /// </summary>
+    public static class LoadDriver
+    {
+        private const string driverInit = "AdbcDriverInit";
+
+        class NativeDriver
+        {
+            public SafeHandle driverHandle;
+            public NativeAdbcDriver driver;
+        }
+
+        /// <summary>
+        /// Class used for Mac interoperability
+        /// </summary>
+        static class MacInterop
+        {
+            const string libdl = "libdl.dylib";
+
+            [DllImport(libdl)]
+            static extern SafeLibraryHandle dlopen(string fileName, int flags);
+
+            [DllImport(libdl)]
+            static extern IntPtr dlsym(SafeHandle libraryHandle, string symbol);
+
+            [DllImport(libdl)]
+            static extern int dlclose(IntPtr handle);
+
+            sealed class SafeLibraryHandle : SafeHandleZeroOrMinusOneIsInvalid
+            {
+                SafeLibraryHandle() : base(true) { }
+
+                protected override bool ReleaseHandle()
+                {
+                    return dlclose(handle) == 0;
+                }
+            }
+
+            public static NativeDriver GetDriver(string file)
+            {
+                SafeHandle library = dlopen(file, 2); // TODO: find a symbol for 2
+                IntPtr symbol = dlsym(library, "AdbcDriverInit");
+                AdbcDriverInit init = Marshal.GetDelegateForFunctionPointer<AdbcDriverInit>(symbol);
+                NativeAdbcDriver driver = new NativeAdbcDriver();
+                NativeAdbcError error = new NativeAdbcError();
+                byte result = init(1000000, ref driver, ref error);
+                return new NativeDriver { driverHandle = library, driver = driver };
+            }
+        }
+
+        /// <summary>
+        /// Class used for Windows interoperability
+        /// </summary>
+        static class WindowsInterop
+        {
+            const string kernel32 = "kernel32.dll";
+
+            [DllImport(kernel32)]
+            [return: MarshalAs(UnmanagedType.Bool)]
+            static extern bool FreeLibrary(IntPtr libraryHandle);
+
+            [DllImport(kernel32, CharSet = CharSet.Ansi, BestFitMapping = false, ThrowOnUnmappableChar = true)]
+            static extern IntPtr GetProcAddress(SafeHandle libraryHandle, string functionName);
+
+            [DllImport(kernel32, CharSet = CharSet.Unicode, SetLastError = true)]
+            static extern SafeLibraryHandle LoadLibraryEx(string fileName, IntPtr hFile, uint flags);
+
+            sealed class SafeLibraryHandle : SafeHandleZeroOrMinusOneIsInvalid
+            {
+                SafeLibraryHandle() : base(true) { }
+
+                protected override bool ReleaseHandle()
+                {
+                    return FreeLibrary(handle);
+                }
+            }
+
+            public static NativeDriver GetDriver(string file)
+            {
+                SafeHandle library = LoadLibraryEx(file, IntPtr.Zero, 0x1100);
+                IntPtr symbol = GetProcAddress(library, "AdbcDriverInit");
+                AdbcDriverInit init = Marshal.GetDelegateForFunctionPointer<AdbcDriverInit>(symbol);
+                NativeAdbcDriver driver = new NativeAdbcDriver();
+                NativeAdbcError error = new NativeAdbcError();
+                byte result = init(1000000, ref driver, ref error);
+                return new NativeDriver { /* driverHandle = library, */ driver = driver };
+            }
+        }
+
+        /// <summary>
+        /// Loads an <see cref="AdbcDriver"/> from the file system.
+        /// </summary>
+        /// <param name="file">The path to the file</param>
+        /// <returns></returns>
+        public static AdbcDriver Load(string file)
+        {
+            if (file[0] == '/')
+            {
+                return new AdbcDriverNative(MacInterop.GetDriver(file).driver);
+            }
+            else
+            {
+                return new AdbcDriverNative(WindowsInterop.GetDriver(file).driver);
+            }
+        }
+
+        /// <summary>
+        /// Native implementation of <see cref="AdbcDriver"/>
+        /// </summary>
+        sealed class AdbcDriverNative : AdbcDriver
+        {
+            private NativeAdbcDriver _nativeDriver;
+
+            public AdbcDriverNative(NativeAdbcDriver nativeDriver)
+            {
+                _nativeDriver = nativeDriver;
+            }
+
+            /// <summary>
+            /// Opens a database
+            /// </summary>
+            /// <param name="parameters"></param>
+            /// <returns></returns>
+            public unsafe override AdbcDatabase Open(Dictionary<string, string> parameters)
+            {
+                NativeAdbcDatabase nativeDatabase = new NativeAdbcDatabase();
+                using (ErrorHelper error = new ErrorHelper())
+                {
+                    error.Call(
+                        Marshal.GetDelegateForFunctionPointer<DatabaseFn>((IntPtr)_nativeDriver.DatabaseNew),
+                        ref nativeDatabase);
+
+                    DatabaseSetOption setOption = Marshal.GetDelegateForFunctionPointer<DatabaseSetOption>((IntPtr)_nativeDriver.DatabaseSetOption);
+                    if (parameters != null)
+                    {
+                        foreach (KeyValuePair<string, string> pair in parameters)
+                        {
+                            error.Call(setOption, ref nativeDatabase, pair.Key, pair.Value);
+                        }
+                    }
+                    error.Call(Marshal.GetDelegateForFunctionPointer<DatabaseFn>((IntPtr)_nativeDriver.DatabaseInit), ref nativeDatabase);
+                }
+
+                return new AdbcDatabaseNative(_nativeDriver, nativeDatabase);
+            }
+
+            public unsafe override void Dispose()
+            {
+                if (_nativeDriver.release != null)
+                {
+                    using (ErrorHelper error = new ErrorHelper())
+                    {
+                        try
+                        {
+                            error.Call(Marshal.GetDelegateForFunctionPointer<DriverRelease>((IntPtr)_nativeDriver.release), ref _nativeDriver);

Review Comment:
   You don't need to / shouldn't use `Marshal.GetDelegateForFunctionPointer` here. Instead you can invoke the function like:
   
   ```C#
   fixed (NativeAdbcDriver* driver = &nativeDriver)
   {
       _nativeDriver.release(driver);
   }
   ```
   
   You'll probably have to rewrite your ErrorHelper for a new pattern. But there's no reason to allocate a managed object (a Delegate) just to call this function - or any native function.



-- 
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-adbc] eerhardt commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1201105004


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcConnection.cs:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.Ipc;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// Provides methods for query execution, managing prepared statements, using transactions, and so on.
+    /// </summary>
+    public abstract class AdbcConnection : IDisposable

Review Comment:
   OK - if the idea/design is to sit an ADO.NET implementation on top of this - then that works for me.



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1215028282


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;
+
+        /// <summary>
+        /// The length of each corresponding entry in partitions.
+        /// </summary>
+        public int* partition_lengths;

Review Comment:
   resolved in latest push



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1211006411


##########
csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/DriverConnectionTests.cs:
##########
@@ -0,0 +1,81 @@
+using System.Collections.Generic;
+using System.IO;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Adbc.FlightSql;
+using Apache.Arrow.Adbc.FlightSql.Tests;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Newtonsoft.Json;
+
+namespace Apache.Arrow.Adbc.Tests
+{
+    [TestClass]
+    public class FlightSqlDriverConnectionTests : DriverConnectionTests
+    {
+        [TestMethod]
+        public override 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();
+
+            long count = 0;
+
+            while (true)
+            {
+                var nextBatch = queryResult.Stream.ReadNextRecordBatchAsync().Result;
+                if (nextBatch == null) { break; }
+                count += nextBatch.Length;
+            }
+
+            Assert.AreEqual(flightSqlTestConfiguration.ExpectedResultsCount, count);
+        }
+    
+        public override void CanDriverUpdate()
+        {
+            throw new System.NotImplementedException();
+        }
+
+        public override void CanReadSchema()
+        {
+            throw new System.NotImplementedException();
+        }
+
+        public override void VerifyBadQueryGeneratesError()
+        {
+            throw new System.NotImplementedException();
+        }
+
+        public override void VerifyTypesAndValues()
+        {
+            throw new System.NotImplementedException();
+        }
+
+        private FlightSqlTestConfiguration GetFlightSqlTestConfiguration()
+        {
+            string json = File.ReadAllText("flightsqlconfig.pass");

Review Comment:
   added a bit more of an explanation. it's setup this way in lieu of doing environment variables. 



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1215026150


##########
csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/.gitignore:
##########
@@ -0,0 +1 @@
+*.pass

Review Comment:
   resolved in latest push



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj:
##########
@@ -0,0 +1,20 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AssemblyName>$(MSBuildProjectName)</AssemblyName>
+    <RootNamespace>Apache.Arrow.Adbc.FlightSql</RootNamespace>

Review Comment:
   resolved in latest push



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1235573463


##########
dev/release/rat_exclude_files.txt:
##########
@@ -30,3 +30,8 @@ c/vendor/sqlite3/sqlite3.c
 c/vendor/sqlite3/sqlite3.h
 *.Rproj
 *.Rd
+csharp/Apache.Arrow.Adbc.sln
+csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj
+csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj
+csharp/test/Apache.Arrow.Adbc.FlightSql.Tests/Apache.Arrow.Adbc.FlightSql.Tests.csproj
+csharp/test/Apache.Arrow.Adbc.Tests/Apache.Arrow.Adbc.Tests.csproj

Review Comment:
   addressing this in https://github.com/apache/arrow-adbc/pull/824 



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1202833175


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcStatement.cs:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.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; }
+
+        public virtual byte[] SubstraitPlan
+        {
+            get { throw new NotImplementedException(); }
+            set { throw new NotImplementedException(); }
+        }
+
+        public virtual void Bind()
+        {
+            throw new NotImplementedException();
+        }
+
+        /// <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()

Review Comment:
   yes. this is the common pattern



-- 
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-adbc] eerhardt commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1202444774


##########
csharp/src/Apache.Arrow.Adbc/Extensions/MatchCollectionExtensions.netstandard.cs:
##########
@@ -0,0 +1,23 @@
+#if NETSTANDARD
+using System.Collections.Generic;
+using System.Linq;
+using System.Text.RegularExpressions;
+
+namespace System.Text.RegularExpressions
+{
+    public static class MatchCollectionExtensions
+    {
+        public static IEnumerable<Match> Where(this MatchCollection matchCollection, Func<Match,bool> predicate)

Review Comment:
   Let's not make this public. You can rewrite the only usage of this method in `ParseDecimalValueFromOverflowException` as:
   
   ```C#
               var matches = regex.Matches(oex.Message);
               foreach (Match match in matches)
               {
                   string value = match.Value;
                   if (!string.IsNullOrEmpty(value))
                   {
                       return value;
                   }
               }
   
               throw oex;
   ```



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1202904992


##########
csharp/src/Apache.Arrow.Adbc/Extensions/MatchCollectionExtensions.netstandard.cs:
##########
@@ -0,0 +1,23 @@
+#if NETSTANDARD
+using System.Collections.Generic;
+using System.Linq;
+using System.Text.RegularExpressions;
+
+namespace System.Text.RegularExpressions
+{
+    public static class MatchCollectionExtensions
+    {
+        public static IEnumerable<Match> Where(this MatchCollection matchCollection, Func<Match,bool> predicate)
+        {
+            List<Match> matches = new List<Match>();
+
+            foreach (Match match in matchCollection) 
+            { 
+                matches.Add(match);
+            }
+            
+            return matches.Where(predicate);

Review Comment:
   removed



##########
csharp/src/Apache.Arrow.Adbc/Extensions/MatchCollectionExtensions.netstandard.cs:
##########
@@ -0,0 +1,23 @@
+#if NETSTANDARD
+using System.Collections.Generic;
+using System.Linq;
+using System.Text.RegularExpressions;
+
+namespace System.Text.RegularExpressions
+{
+    public static class MatchCollectionExtensions
+    {
+        public static IEnumerable<Match> Where(this MatchCollection matchCollection, Func<Match,bool> predicate)

Review Comment:
   removed



-- 
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-adbc] eerhardt commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "eerhardt (via GitHub)" <gi...@apache.org>.
eerhardt commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1201105004


##########
csharp/src/Apache.Arrow.Adbc/Core/AdbcConnection.cs:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.Ipc;
+
+namespace Apache.Arrow.Adbc.Core
+{
+    /// <summary>
+    /// Provides methods for query execution, managing prepared statements, using transactions, and so on.
+    /// </summary>
+    public abstract class AdbcConnection : IDisposable

Review Comment:
   OK - if the idea/design is to sit an ADO.NET implementation on top of this - then that works for me.
   
   We can add it if/when we get feedback for it. Doesn't need to be part of this PR.



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

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

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


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

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1205823912


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,608 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    public static class AdbcInterop
+    {
+        private static unsafe NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);

Review Comment:
   discussed over chat that these need to remain unsafe because the method it is referencing has pointers in it



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

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

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


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

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1205826614


##########
csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj:
##########
@@ -0,0 +1,18 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <Compile Remove="Extensions\MatchCollectionExtensions.netstandard.cs" />

Review Comment:
   no. this was removed.



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1213164673


##########
csharp/src/Apache.Arrow.Adbc/Apache.Arrow.Adbc.csproj:
##########
@@ -0,0 +1,13 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <ProjectReference Include="..\arrow\csharp\src\Apache.Arrow\Apache.Arrow.csproj" />

Review Comment:
   uses a submodule to obtain. will investigate gh action suggestion



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1218101842


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;
+            //nativeDriver->ConnectionGetTableSchema = null;
+            //nativeDriver->ConnectionGetTableTypes = null;
+            nativeDriver->ConnectionInit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)connectionInit.Pointer;
+            nativeDriver->ConnectionNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)stub.newConnection.Pointer;
+            nativeDriver->ConnectionSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>)connectionSetOption.Pointer;
+            //nativeDriver->ConnectionReadPartition = null;
+            nativeDriver->ConnectionRelease = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionRollback = null;
+
+           // nativeDriver->StatementBind = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArray*, CArrowSchema*, NativeAdbcError*, AdbcStatusCode>)
+            nativeDriver->StatementNew = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementNew.Pointer;
+            nativeDriver->StatementSetSqlQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, byte*, NativeAdbcError *, AdbcStatusCode >)statementSetSqlQuery.Pointer;
+            nativeDriver->StatementExecuteQuery = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, CArrowArrayStream*, long*, NativeAdbcError*, AdbcStatusCode>)statementExecuteQuery.Pointer;
+            nativeDriver->StatementPrepare = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            nativeDriver->StatementRelease = (delegate* unmanaged[Stdcall]<NativeAdbcStatement*, NativeAdbcError*, AdbcStatusCode>)statementRelease.Pointer;
+            
+            return 0;
+        }
+
+        private unsafe static void ReleaseError(NativeAdbcError* error)
+        {
+            if (error != null && ((IntPtr)error->message) != IntPtr.Zero)
+            {
+                Marshal.FreeCoTaskMem((IntPtr)error->message);
+            }
+        }
+
+        private unsafe static AdbcStatusCode SetError(NativeAdbcError* error, Exception exception)
+        {
+            ReleaseError(error);
+
+            #if NETSTANDARD
+                error->message = (char*)MarshalExtensions.StringToCoTaskMemUTF8(exception.Message);
+            #else
+                error->message = (char*)Marshal.StringToCoTaskMemUTF8(exception.Message);
+            #endif
+
+            error->sqlstate0 = (char)0;
+            error->sqlstate1 = (char)0;
+            error->sqlstate2 = (char)0;
+            error->sqlstate3 = (char)0;
+            error->sqlstate4 = (char)0;
+            error->vendor_code = 0;
+            error->vendor_code = 0;
+            error->release = (delegate* unmanaged[Stdcall]<NativeAdbcError*, void>)releaseError.Pointer;
+            
+            return AdbcStatusCode.UnknownError;
+        }
+
+        private sealed class PinnedArray : IDisposable
+        {
+            IArrowArray _array;
+            MemoryHandle[] pinnedHandles;
+
+            public PinnedArray(IArrowArray array)
+            {
+                _array = array;
+                pinnedHandles = new MemoryHandle[GetHandleCount(array.Data)];
+                int index = 0;
+                PinBuffers(array.Data, pinnedHandles, ref index);
+                Debug.Assert(index == pinnedHandles.Length);
+            }
+
+            public void Dispose()
+            {
+                if (_array != null)
+                {
+                    _array.Dispose();
+                    foreach (MemoryHandle handle in pinnedHandles)
+                    {
+                        handle.Dispose();
+                    }
+                    _array = null;
+                }
+            }
+
+            static int GetHandleCount(ArrayData data)
+            {
+                int handleCount = data.Buffers.Length;
+                foreach (ArrayData child in data.Children)
+                {
+                    handleCount += GetHandleCount(child);
+                }
+                if (data.Dictionary != null)
+                {
+                    handleCount += GetHandleCount(data.Dictionary);
+                }
+                return handleCount;
+            }
+
+            static void PinBuffers(ArrayData data, MemoryHandle[] handles, ref int index)
+            {
+                foreach (ArrowBuffer buffer in data.Buffers)
+                {
+                    handles[index++] = buffer.Memory.Pin();
+                }
+                foreach (ArrayData child in data.Children)
+                {
+                    PinBuffers(child, handles, ref index);
+                }
+                if (data.Dictionary != null)
+                {
+                    PinBuffers(data.Dictionary, handles, ref index);
+                }
+            }
+        }
+
+        private static IntPtr FromDisposable(IDisposable d)
+        {
+            GCHandle gch = GCHandle.Alloc(d);
+            return GCHandle.ToIntPtr(gch);
+        }
+
+        private static void Dispose(ref IntPtr p)
+        {
+            GCHandle gch = GCHandle.FromIntPtr(p);
+            ((IDisposable)gch.Target).Dispose();
+            gch.Free();
+            p = IntPtr.Zero;
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDriver(NativeAdbcDriver* nativeDriver, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDriver->private_data);
+            DriverStub stub = (DriverStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDriver->private_data = null;
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode InitDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            return stub.Init(ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseDatabase(NativeAdbcDatabase* nativeDatabase, NativeAdbcError* error)
+        {
+            if (nativeDatabase->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeDatabase->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode SetConnectionOption(NativeAdbcConnection* nativeConnection, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetDatabaseOption(NativeAdbcDatabase* nativeDatabase, byte* name, byte* value, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeDatabase->private_data);
+            DatabaseStub stub = (DatabaseStub)gch.Target;
+
+            return stub.SetOption(name, value, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode InitConnection(NativeAdbcConnection* nativeConnection, NativeAdbcDatabase* database, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.InitConnection(ref *database, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseConnection(NativeAdbcConnection* nativeConnection, NativeAdbcError* error)
+        {
+            if (nativeConnection->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeConnection->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode GetConnectionInfo(NativeAdbcConnection* nativeConnection, uint* info_codes, int info_codes_length, CArrowArrayStream* stream, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.GetConnectionInfo(ref *nativeConnection, *info_codes, info_codes_length, ref *stream, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode SetStatementSqlQuery(NativeAdbcStatement* nativeStatement, byte* text, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+
+            #if NETSTANDARD
+                stub.SqlQuery = MarshalExtensions.PtrToStringUTF8((IntPtr)text);
+            #else
+                stub.SqlQuery = Marshal.PtrToStringUTF8((IntPtr)text);
+            #endif
+
+            return AdbcStatusCode.Success;
+        }
+
+        private unsafe static AdbcStatusCode ExecuteStatementQuery(NativeAdbcStatement* nativeStatement, CArrowArrayStream* stream, long* rows, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            var result = stub.ExecuteQuery();
+            if (rows != null)
+            {
+                *rows = result.RowCount;
+            }
+
+            GCHandle streamHandle = GCHandle.Alloc(result.Stream);
+            stream->private_data = (void*)GCHandle.ToIntPtr(streamHandle);
+
+            return 0;
+        }
+
+        private unsafe static AdbcStatusCode NewStatement(NativeAdbcConnection* nativeConnection, NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeConnection->private_data);
+            ConnectionStub stub = (ConnectionStub)gch.Target;
+            return stub.NewStatement(ref *nativeStatement, ref *error);
+        }
+
+        private unsafe static AdbcStatusCode ReleaseStatement(NativeAdbcStatement* nativeStatement, NativeAdbcError* error)
+        {
+            if (nativeStatement->private_data == null)
+            {
+                return AdbcStatusCode.UnknownError;
+            }
+
+            GCHandle gch = GCHandle.FromIntPtr((IntPtr)nativeStatement->private_data);
+            AdbcStatement stub = (AdbcStatement)gch.Target;
+            stub.Dispose();
+            gch.Free();
+            nativeStatement->private_data = null;
+            return AdbcStatusCode.Success;
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcDatabase
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+
+        public static NativeAdbcDatabase* Create()
+        {
+            var ptr = (NativeAdbcDatabase*)Marshal.AllocHGlobal(sizeof(NativeAdbcDatabase));
+
+            ptr->private_data = null;
+            ptr->private_driver = null;
+            
+            return ptr;
+        }
+
+        /// <summary>
+        /// Free a pointer that was allocated in <see cref="Create"/>.
+        /// </summary>
+        /// <remarks>
+        /// Do not call this on a pointer that was allocated elsewhere.
+        /// </remarks>
+        public static void Free(NativeAdbcDatabase* database)
+        {
+            Marshal.FreeHGlobal((IntPtr)database);
+        }
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcConnection
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcStatement
+    {
+        public void* private_data;
+        public NativeAdbcDriver* private_driver;
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcPartitions
+    {
+        /// <summary>
+        /// The number of partitions.
+        /// </summary>
+        public int num_partitions;
+
+        /// <summary>
+        /// The partitions of the result set, where each entry (up to
+        /// num_partitions entries) is an opaque identifier that can be
+        /// passed to AdbcConnectionReadPartition.
+        /// </summary>
+        public sbyte** partitions;
+
+        /// <summary>
+        /// The length of each corresponding entry in partitions.
+        /// </summary>
+        public int* partition_lengths;
+
+        /// <summary>
+        /// Opaque implementation-defined state.
+        /// This field is NULLPTR iff the connection is unintialized/freed.
+        /// </summary>
+        public void* private_data;
+
+        /// <summary>
+        /// Release the contained partitions.
+        ///
+        /// Unlike other structures, this is an embedded callback to make it
+        /// easier for the driver manager and driver to cooperate.
+        /// </summary>
+        public delegate* unmanaged[Stdcall]<NativeAdbcPartitions*, void> release; 
+    }
+
+    [StructLayout(LayoutKind.Sequential)]
+    public unsafe struct NativeAdbcError
+    {
+        /// <summary>
+        /// The error message.
+        /// </summary>
+        public char* message;
+
+        /// <summary>
+        /// A vendor-specific error code, if applicable.
+        /// </summary>
+        public int vendor_code;
+
+        /// <summary>
+        /// A SQLSTATE error code, if provided, as defined by the
+        ///   SQL:2003 standard.  If not set, it should be set to
+        ///   "\0\0\0\0\0".
+        ///</summary>
+        public char sqlstate0;
+        public char sqlstate1;
+        public char sqlstate2;
+        public char sqlstate3;
+        public char sqlstate4;

Review Comment:
   verified



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1218458861


##########
csharp/src/Apache.Arrow.Adbc/Core/Interop.cs:
##########
@@ -0,0 +1,694 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Runtime.InteropServices;
+using Apache.Arrow.Adbc.Interop;
+using Apache.Arrow.C;
+using static System.Net.Mime.MediaTypeNames;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Core
+{
+    internal static class AdbcInterop
+    {
+        private unsafe static readonly NativeDelegate<ErrorRelease> releaseError = new NativeDelegate<ErrorRelease>(ReleaseError);
+        private unsafe static readonly NativeDelegate<DriverRelease> releaseDriver = new NativeDelegate<DriverRelease>(ReleaseDriver);
+
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseInit = new NativeDelegate<DatabaseFn>(InitDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseFn> databaseRelease = new NativeDelegate<DatabaseFn>(ReleaseDatabase);
+        private unsafe static readonly NativeDelegate<DatabaseSetOption> databaseSetOption = new NativeDelegate<DatabaseSetOption>(SetDatabaseOption);
+
+        private unsafe static readonly NativeDelegate<ConnectionInit> connectionInit = new NativeDelegate<ConnectionInit>(InitConnection);
+        private unsafe static readonly NativeDelegate<ConnectionFn> connectionRelease = new NativeDelegate<ConnectionFn>(ReleaseConnection);
+        private unsafe static readonly NativeDelegate<ConnectionGetInfo> connectionGetInfo = new NativeDelegate<ConnectionGetInfo>(GetConnectionInfo);
+        private unsafe static readonly NativeDelegate<ConnectionSetOption> connectionSetOption = new NativeDelegate<ConnectionSetOption>(SetConnectionOption);
+        
+        private unsafe static readonly NativeDelegate<StatementExecuteQuery> statementExecuteQuery = new NativeDelegate<StatementExecuteQuery>(ExecuteStatementQuery);
+        private unsafe static readonly NativeDelegate<StatementNew> statementNew = new NativeDelegate<StatementNew>(NewStatement);
+        private unsafe static readonly NativeDelegate<StatementFn> statementRelease = new NativeDelegate<StatementFn>(ReleaseStatement);
+        private unsafe static readonly NativeDelegate<StatementSetSqlQuery> statementSetSqlQuery = new NativeDelegate<StatementSetSqlQuery>(SetStatementSqlQuery);
+
+        public unsafe static AdbcStatusCode AdbcDriverInit(int version, NativeAdbcDriver* nativeDriver, NativeAdbcError* error, AdbcDriver driver)
+        {
+            DriverStub stub = new DriverStub(driver);
+            GCHandle handle = GCHandle.Alloc(stub);
+            nativeDriver->private_data = (void*)GCHandle.ToIntPtr(handle);
+            nativeDriver->release = (delegate* unmanaged[Stdcall]<NativeAdbcDriver*, NativeAdbcError*, AdbcStatusCode>)releaseDriver.Pointer;
+
+            nativeDriver->DatabaseInit = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseInit.Pointer;
+            nativeDriver->DatabaseNew = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)stub.newDatabase.Pointer;
+            nativeDriver->DatabaseSetOption = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, byte*, byte*, NativeAdbcError*, AdbcStatusCode>) databaseSetOption.Pointer;
+            nativeDriver->DatabaseRelease = (delegate* unmanaged[Stdcall]<NativeAdbcDatabase*, NativeAdbcError*, AdbcStatusCode>)databaseRelease.Pointer;
+
+            nativeDriver->ConnectionCommit = (delegate* unmanaged[Stdcall]<NativeAdbcConnection*, NativeAdbcError*, AdbcStatusCode>)connectionRelease.Pointer;
+            //nativeDriver->ConnectionGetInfo = (delegate* unmanaged[Stdcall]<NativeAdbcConnection *, int*, int, CArrowArrayStream*, NativeAdbcError*, AdbcStatusCode>)connectionGetInfo.Pointer;

Review Comment:
   these are filled out now



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

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

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


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

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1214605789


##########
csharp/src/Apache.Arrow.Adbc/Interop/LoadDriver.cs:
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Buffers;
+using System.Collections.Generic;
+using System.Runtime.InteropServices;
+using System.Threading;
+using System.Threading.Tasks;
+using System.Xml.Linq;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.C;
+using Apache.Arrow.Ipc;
+using Apache.Arrow.Types;
+using Microsoft.Win32.SafeHandles;
+
+#if NETSTANDARD
+using Apache.Arrow.Adbc.Extensions;
+#endif
+
+namespace Apache.Arrow.Adbc.Interop
+{
+    public delegate byte AdbcDriverInit(int version, ref NativeAdbcDriver driver, ref NativeAdbcError error);
+
+    /// <summary>
+    /// Class for working with loading drivers from files
+    /// </summary>
+    public static class LoadDriver
+    {
+        private const string driverInit = "AdbcDriverInit";
+
+        class NativeDriver
+        {
+            public SafeHandle driverHandle;
+            public NativeAdbcDriver driver;
+        }
+
+        /// <summary>
+        /// Class used for Mac interoperability
+        /// </summary>
+        static class MacInterop
+        {
+            const string libdl = "libdl.dylib";
+
+            [DllImport(libdl)]
+            static extern SafeLibraryHandle dlopen(string fileName, int flags);
+
+            [DllImport(libdl)]
+            static extern IntPtr dlsym(SafeHandle libraryHandle, string symbol);
+
+            [DllImport(libdl)]
+            static extern int dlclose(IntPtr handle);
+
+            sealed class SafeLibraryHandle : SafeHandleZeroOrMinusOneIsInvalid
+            {
+                SafeLibraryHandle() : base(true) { }
+
+                protected override bool ReleaseHandle()
+                {
+                    return dlclose(handle) == 0;
+                }
+            }
+
+            public static NativeDriver GetDriver(string file)
+            {
+                SafeHandle library = dlopen(file, 2); // TODO: find a symbol for 2
+                IntPtr symbol = dlsym(library, "AdbcDriverInit");
+                AdbcDriverInit init = Marshal.GetDelegateForFunctionPointer<AdbcDriverInit>(symbol);
+                NativeAdbcDriver driver = new NativeAdbcDriver();
+                NativeAdbcError error = new NativeAdbcError();
+                byte result = init(1000000, ref driver, ref error);
+                return new NativeDriver { driverHandle = library, driver = driver };
+            }
+        }
+
+        /// <summary>
+        /// Class used for Windows interoperability
+        /// </summary>
+        static class WindowsInterop
+        {
+            const string kernel32 = "kernel32.dll";
+
+            [DllImport(kernel32)]
+            [return: MarshalAs(UnmanagedType.Bool)]
+            static extern bool FreeLibrary(IntPtr libraryHandle);
+
+            [DllImport(kernel32, CharSet = CharSet.Ansi, BestFitMapping = false, ThrowOnUnmappableChar = true)]
+            static extern IntPtr GetProcAddress(SafeHandle libraryHandle, string functionName);
+
+            [DllImport(kernel32, CharSet = CharSet.Unicode, SetLastError = true)]
+            static extern SafeLibraryHandle LoadLibraryEx(string fileName, IntPtr hFile, uint flags);
+
+            sealed class SafeLibraryHandle : SafeHandleZeroOrMinusOneIsInvalid
+            {
+                SafeLibraryHandle() : base(true) { }
+
+                protected override bool ReleaseHandle()
+                {
+                    return FreeLibrary(handle);
+                }
+            }
+
+            public static NativeDriver GetDriver(string file)
+            {
+                SafeHandle library = LoadLibraryEx(file, IntPtr.Zero, 0x1100);
+                IntPtr symbol = GetProcAddress(library, "AdbcDriverInit");
+                AdbcDriverInit init = Marshal.GetDelegateForFunctionPointer<AdbcDriverInit>(symbol);
+                NativeAdbcDriver driver = new NativeAdbcDriver();
+                NativeAdbcError error = new NativeAdbcError();
+                byte result = init(1000000, ref driver, ref error);
+                return new NativeDriver { /* driverHandle = library, */ driver = driver };
+            }
+        }
+
+        /// <summary>
+        /// Loads an <see cref="AdbcDriver"/> from the file system.
+        /// </summary>
+        /// <param name="file">The path to the file</param>
+        /// <returns></returns>
+        public static AdbcDriver Load(string file)
+        {
+            if (file[0] == '/')
+            {
+                return new AdbcDriverNative(MacInterop.GetDriver(file).driver);
+            }
+            else
+            {
+                return new AdbcDriverNative(WindowsInterop.GetDriver(file).driver);
+            }
+        }
+
+        /// <summary>
+        /// Native implementation of <see cref="AdbcDriver"/>
+        /// </summary>
+        sealed class AdbcDriverNative : AdbcDriver
+        {
+            private NativeAdbcDriver _nativeDriver;
+
+            public AdbcDriverNative(NativeAdbcDriver nativeDriver)
+            {
+                _nativeDriver = nativeDriver;
+            }
+
+            /// <summary>
+            /// Opens a database
+            /// </summary>
+            /// <param name="parameters"></param>
+            /// <returns></returns>
+            public unsafe override AdbcDatabase Open(Dictionary<string, string> parameters)
+            {
+                NativeAdbcDatabase nativeDatabase = new NativeAdbcDatabase();
+                using (ErrorHelper error = new ErrorHelper())
+                {
+                    error.Call(
+                        Marshal.GetDelegateForFunctionPointer<DatabaseFn>((IntPtr)_nativeDriver.DatabaseNew),
+                        ref nativeDatabase);
+
+                    DatabaseSetOption setOption = Marshal.GetDelegateForFunctionPointer<DatabaseSetOption>((IntPtr)_nativeDriver.DatabaseSetOption);
+                    if (parameters != null)
+                    {
+                        foreach (KeyValuePair<string, string> pair in parameters)
+                        {
+                            error.Call(setOption, ref nativeDatabase, pair.Key, pair.Value);
+                        }
+                    }
+                    error.Call(Marshal.GetDelegateForFunctionPointer<DatabaseFn>((IntPtr)_nativeDriver.DatabaseInit), ref nativeDatabase);
+                }
+
+                return new AdbcDatabaseNative(_nativeDriver, nativeDatabase);
+            }
+
+            public unsafe override void Dispose()
+            {
+                if (_nativeDriver.release != null)
+                {
+                    using (ErrorHelper error = new ErrorHelper())
+                    {
+                        try
+                        {
+                            error.Call(Marshal.GetDelegateForFunctionPointer<DriverRelease>((IntPtr)_nativeDriver.release), ref _nativeDriver);

Review Comment:
   done in the latest push



-- 
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-adbc] davidhcoe commented on pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1581449736

   chatted with Eric on the side -- he is good to close this PR and incorporate his comments in to a different PR. I've noted them for tracking.


-- 
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-adbc] lidavidm commented on pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#issuecomment-1581486442

   Thanks @davidhcoe @eerhardt @CurtHagenlocher!


-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220824728


##########
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:
   moved to IReadOnlyDictionary



-- 
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-adbc] davidhcoe commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "davidhcoe (via GitHub)" <gi...@apache.org>.
davidhcoe commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220823011


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

Review Comment:
   ah, interesting. yes. It turns out that a root level .gitignore was preventing the one I had from being added



-- 
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-adbc] lidavidm commented on a diff in pull request #697: feat(csharp): adding C# functionality

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #697:
URL: https://github.com/apache/arrow-adbc/pull/697#discussion_r1220194997


##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache - Backup.Arrow.Adbc.FlightSql.csproj:
##########


Review Comment:
   Did you mean to include this 'backup' file?



##########
csharp/src/Apache.Arrow.Adbc/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
+{
+    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:
   Is there a need to override this then? (Or should it take into account `_descriptor`?)



##########
csharp/src/Apache.Arrow.Adbc/AdbcStatement.cs:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.Text.RegularExpressions;
+using System.Threading.Tasks;
+using Apache.Arrow.Ipc;
+
+namespace Apache.Arrow.Adbc
+{
+    /// <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()
+        {
+            
+        }
+
+        /// <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>
+        /// Execute a result set-generating query and get a list of partitions of the result set.
+        /// </summary>
+        /// <returns><see cref="PartitionedResult"/></returns>
+        public virtual PartitionedResult ExecutePartitioned()
+        {
+            throw AdbcException.NotImplemented("Statement does not support ExecutePartitioned");
+        }
+
+        /// <summary>
+        /// Get the schema for bound parameters.
+        /// </summary>
+        /// <returns><see cref="Schema"/></returns>
+        public virtual Schema GetParameterSchema()
+        {
+            throw AdbcException.NotImplemented("Statement does not support GetParameterSchema");
+        }
+
+        /// <summary>
+        ///  Turn this statement into a prepared statement to be
+        ///  executed multiple times.
+        /// </summary>
+        public virtual void Prepare()
+        {
+            throw AdbcException.NotImplemented("Statement does not support Prepare");
+        }
+
+        public virtual void Dispose()
+        {
+        }
+
+        /// <summary>
+        /// Gets a value from the Arrow array at the specified index, using the Field metadata for information.
+        /// </summary>
+        /// <param name="arrowArray">The Arrow array.</param>
+        /// <param name="field">The <see cref="Field"/> from the <see cref="Schema"/> that can be used for metadata inspection.</param>
+        /// <param name="index">The index in the array to get the value from.</param>
+        /// <returns></returns>
+        public abstract object GetValue(IArrowArray arrowArray, Field field, int index);
+
+        /// <summary>
+        /// For decimals, Arrow throws an OverflowException if a value is < decimal.min or > decimal.max
+        /// So parse the numeric value and return it as a string, if possible
+        /// </summary>
+        /// <param name="oex"></param>
+        /// <returns>A string value of the decimal that threw the exception or rethrows the OverflowException.</returns>
+        /// <exception cref="ArgumentNullException"></exception>
+        public virtual string ParseDecimalValueFromOverflowException(OverflowException oex)

Review Comment:
   This still doesn't feel like it belongs in the public API, at least not as part of a Statement object



##########
csharp/src/Apache.Arrow.Adbc/AdbcConnection.cs:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.Ipc;
+
+namespace Apache.Arrow.Adbc
+{
+    /// <summary>
+    /// Provides methods for query execution, managing prepared statements, using transactions, and so on.
+    /// </summary>
+    public abstract class AdbcConnection : IDisposable
+    {
+        private bool _autoCommit = true;
+        private bool _readOnly = false;
+        private IsolationLevel _isolationLevel = IsolationLevel.Default;

Review Comment:
   These might have to be `protected` for implementors?



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlStatement.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.Threading.Tasks;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Flight;
+using Grpc.Core;
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// A Flight SQL implementation of <see cref="AdbcStatement"/>.
+    /// </summary>
+    public class FlightSqlStatement : AdbcStatement
+    {
+        private FlightSqlConnection flightSqlConnection;
+        
+        public FlightSqlStatement(FlightSqlConnection flightSqlConnection)
+        {
+            this.flightSqlConnection = flightSqlConnection;
+        }
+
+        public override async ValueTask<QueryResult> ExecuteQueryAsync()
+        {
+            FlightInfo info = await GetInfo(this.SqlQuery, this.flightSqlConnection.Metadata);
+
+            return new QueryResult(info.TotalRecords, new FlightSqlResult(this.flightSqlConnection, info));
+        }
+
+        public override QueryResult ExecuteQuery()
+        {
+            return ExecuteQueryAsync().Result;
+        }
+
+        public override UpdateResult ExecuteUpdate()
+        {
+            throw new NotImplementedException();
+        }
+
+        public async ValueTask<FlightInfo> GetInfo(string query, Metadata headers)
+        {
+            FlightDescriptor commandDescripter = FlightDescriptor.CreateCommandDescriptor(query);
+
+            return await this.flightSqlConnection.FlightClient.GetInfo(commandDescripter, headers).ResponseAsync;
+        }
+
+        /// <summary>
+        /// Gets a value from the Arrow array at the specified index using the Arrow field for metadata.
+        /// </summary>
+        /// <param name="arrowArray"></param>
+        /// <param name="field"></param>
+        /// <param name="index"></param>
+        /// <returns></returns>
+        public override object GetValue(IArrowArray arrowArray, Field field, int index)

Review Comment:
   I'm still skeptical that the way to deal with this is to provide accessors for each driver, rather than having the driver convert things to the proper Arrow type in the first place.



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlStatement.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.Threading.Tasks;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Flight;
+using Grpc.Core;
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// A Flight SQL implementation of <see cref="AdbcStatement"/>.
+    /// </summary>
+    public class FlightSqlStatement : AdbcStatement
+    {
+        private FlightSqlConnection flightSqlConnection;
+        
+        public FlightSqlStatement(FlightSqlConnection flightSqlConnection)
+        {
+            this.flightSqlConnection = flightSqlConnection;
+        }
+
+        public override async ValueTask<QueryResult> ExecuteQueryAsync()
+        {
+            FlightInfo info = await GetInfo(this.SqlQuery, this.flightSqlConnection.Metadata);
+
+            return new QueryResult(info.TotalRecords, new FlightSqlResult(this.flightSqlConnection, info));
+        }
+
+        public override QueryResult ExecuteQuery()
+        {
+            return ExecuteQueryAsync().Result;
+        }
+
+        public override UpdateResult ExecuteUpdate()
+        {
+            throw new NotImplementedException();
+        }
+
+        public async ValueTask<FlightInfo> GetInfo(string query, Metadata headers)
+        {
+            FlightDescriptor commandDescripter = FlightDescriptor.CreateCommandDescriptor(query);
+
+            return await this.flightSqlConnection.FlightClient.GetInfo(commandDescripter, headers).ResponseAsync;
+        }
+
+        /// <summary>
+        /// Gets a value from the Arrow array at the specified index using the Arrow field for metadata.
+        /// </summary>
+        /// <param name="arrowArray"></param>
+        /// <param name="field"></param>
+        /// <param name="index"></param>
+        /// <returns></returns>
+        public override object GetValue(IArrowArray arrowArray, Field field, int index)

Review Comment:
   At least, this implementation could be used as the default implementation, and then Flight SQL shouldn't need to override anything and other drivers could use it as a starting point?



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/FlightSqlStatement.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.Threading.Tasks;
+using Apache.Arrow.Adbc.Core;
+using Apache.Arrow.Flight;
+using Grpc.Core;
+
+namespace Apache.Arrow.Adbc.FlightSql
+{
+    /// <summary>
+    /// A Flight SQL implementation of <see cref="AdbcStatement"/>.
+    /// </summary>
+    public class FlightSqlStatement : AdbcStatement
+    {
+        private FlightSqlConnection flightSqlConnection;
+        
+        public FlightSqlStatement(FlightSqlConnection flightSqlConnection)
+        {
+            this.flightSqlConnection = flightSqlConnection;
+        }
+
+        public override async ValueTask<QueryResult> ExecuteQueryAsync()
+        {
+            FlightInfo info = await GetInfo(this.SqlQuery, this.flightSqlConnection.Metadata);
+
+            return new QueryResult(info.TotalRecords, new FlightSqlResult(this.flightSqlConnection, info));
+        }
+
+        public override QueryResult ExecuteQuery()
+        {
+            return ExecuteQueryAsync().Result;
+        }
+
+        public override UpdateResult ExecuteUpdate()
+        {
+            throw new NotImplementedException();
+        }
+
+        public async ValueTask<FlightInfo> GetInfo(string query, Metadata headers)
+        {
+            FlightDescriptor commandDescripter = FlightDescriptor.CreateCommandDescriptor(query);
+
+            return await this.flightSqlConnection.FlightClient.GetInfo(commandDescripter, headers).ResponseAsync;
+        }
+
+        /// <summary>
+        /// Gets a value from the Arrow array at the specified index using the Arrow field for metadata.
+        /// </summary>
+        /// <param name="arrowArray"></param>
+        /// <param name="field"></param>
+        /// <param name="index"></param>
+        /// <returns></returns>
+        public override object GetValue(IArrowArray arrowArray, Field field, int index)

Review Comment:
   Though it's also possible that the C♯ library lacks efficient/convenient ways to implement such conversions (as compared to Go or C++).
   
   Again at least for Snowflake the ADBC Go driver already does these conversions.



##########
csharp/src/Apache.Arrow.Adbc.FlightSql/Apache.Arrow.Adbc.FlightSql.csproj:
##########
@@ -0,0 +1,20 @@
+<Project Sdk="Microsoft.NET.Sdk">
+
+  <PropertyGroup>
+    <TargetFrameworks>netstandard2.0;net6.0</TargetFrameworks>
+    <AssemblyName>$(MSBuildProjectName)</AssemblyName>
+    <RootNamespace>Apache.Arrow.Adbc.FlightSql</RootNamespace>
+    <Version>0.1.0</Version>
+  </PropertyGroup>
+
+  <ItemGroup>
+    <PackageReference Include="Apache.Arrow.Flight" Version="12.0.0" />
+    <PackageReference Include="Grpc.Net.Client.Web" Version="2.53.0" />
+    <PackageReference Include="System.Net.Http.WinHttpHandler" Version="7.0.0" />

Review Comment:
   Seems this wasn't addressed?



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