You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2022/12/13 09:36:28 UTC

[ignite-3] branch main updated: IGNITE-18136 .NET: LINQ: Emit and cache deserialization methods (#1439)

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

ptupitsyn pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 76b414bca0 IGNITE-18136 .NET: LINQ: Emit and cache deserialization methods (#1439)
76b414bca0 is described below

commit 76b414bca00a43981da83a5200486433e8e38766
Author: Pavel Tupitsyn <pt...@apache.org>
AuthorDate: Tue Dec 13 12:36:22 2022 +0300

    IGNITE-18136 .NET: LINQ: Emit and cache deserialization methods (#1439)
    
    * Extract LINQ result selector logic to `ResultSelector` class.
    * Replace naive and inefficient reflection with compiled and cached methods.
      * Cache key includes column metadata: if table or query gets altered (columns added/removed, type changed), different logic is required.
    * Handle outer joins (`DefaultIfEmpty`) - read nulls as default values for value types.
    
    As a result, LINQ performance is on par with SQL. Allocated memory is even lower, because `IgniteTuple` is not used to store results - we read directly into user class or primitive types.
    
    ```
    |                         Method |     Mean |     Error |    StdDev | Ratio |   Gen 0 | Allocated |
    |------------------------------- |---------:|----------:|----------:|------:|--------:|----------:|
    |                    ToListAsync | 2.325 ms | 0.0207 ms | 0.0183 ms |  1.00 | 15.6250 |    392 KB |
    |                AsyncEnumerable | 2.487 ms | 0.0202 ms | 0.0189 ms |  1.07 | 15.6250 |    393 KB |
    |                LinqToListAsync | 2.206 ms | 0.0121 ms | 0.0113 ms |  0.95 |       - |     80 KB |
    | LinqSelectOneColumnToListAsync | 2.171 ms | 0.0278 ms | 0.0246 ms |  0.93 |       - |     57 KB |
    ```
---
 .../dotnet/Apache.Ignite.Benchmarks/.editorconfig  |   1 +
 .../dotnet/Apache.Ignite.Benchmarks/Program.cs     |  17 +-
 .../Sql/ResultSetBenchmarks.cs                     |  51 +++-
 .../Apache.Ignite.Tests/Apache.Ignite.Tests.csproj |   4 +-
 .../dotnet/Apache.Ignite.Tests/IgniteTestsBase.cs  |   7 +-
 .../Apache.Ignite.Tests/Linq/LinqTests.Join.cs     |  41 ++-
 .../Apache.Ignite.Tests/Linq/LinqTests.KvView.cs   |  62 +++++
 .../dotnet/Apache.Ignite.Tests/Linq/LinqTests.cs   |  32 +++
 .../Linq/ResultSelectorCacheKeyTests.cs            | 131 +++++++++
 .../Table/PocoAllColumnsNullable.cs                |  48 ++++
 .../Table/RecordViewPocoTests.cs                   |  28 ++
 .../platforms/dotnet/Apache.Ignite.sln.DotSettings |   1 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj      |   4 +-
 .../Internal/Linq/IgniteQueryExecutor.cs           | 101 +------
 .../Internal/Linq/IgniteQueryModelVisitor.cs       |  10 +-
 .../Apache.Ignite/Internal/Linq/QueryData.cs       |   2 +-
 .../Apache.Ignite/Internal/Linq/ResultSelector.cs  | 305 +++++++++++++++++++++
 .../Internal/Linq/ResultSelectorCacheKey.cs        | 123 +++++++++
 .../Proto/BinaryTuple/BinaryTupleReader.cs         |   7 +
 .../dotnet/Apache.Ignite/Internal/Sql/Sql.cs       |   2 +-
 .../Table/Serialization/BinaryTupleMethods.cs      |   7 +
 .../Table/Serialization/ILGeneratorExtensions.cs   |  78 ++++++
 .../Table/Serialization/ObjectSerializerHandler.cs |  93 +------
 23 files changed, 946 insertions(+), 209 deletions(-)

diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/.editorconfig b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/.editorconfig
index 4a85312152..ab4bf71a2e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/.editorconfig
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/.editorconfig
@@ -23,4 +23,5 @@ dotnet_diagnostic.CA1014.severity = none # Mark assemblies with CLSCompliant
 dotnet_diagnostic.CA1720.severity = none # Identifier contains type name
 dotnet_diagnostic.CA2201.severity = none # Do not raise reserved exception types
 dotnet_diagnostic.CA1001.severity = none # Types that own disposable fields should be disposable
+dotnet_diagnostic.CA1812.severity = none # Class is never instantiated
 
diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Program.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Program.cs
index 475523ec21..04fd5dde56 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Program.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Program.cs
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-namespace Apache.Ignite.Benchmarks
-{
-    using BenchmarkDotNet.Running;
-    using Table.Serialization;
+namespace Apache.Ignite.Benchmarks;
+
+using BenchmarkDotNet.Running;
+using Sql;
 
-    internal static class Program
+internal static class Program
+{
+    private static void Main()
     {
-        private static void Main()
-        {
-            BenchmarkRunner.Run<SerializerHandlerReadBenchmarks>();
-        }
+        BenchmarkRunner.Run<ResultSetBenchmarks>();
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Sql/ResultSetBenchmarks.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Sql/ResultSetBenchmarks.cs
index 1a7e9ef90b..fc7516190e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Sql/ResultSetBenchmarks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Sql/ResultSetBenchmarks.cs
@@ -19,36 +19,41 @@ namespace Apache.Ignite.Benchmarks.Sql
 {
     using System;
     using System.Collections.Generic;
+    using System.Linq;
     using System.Threading.Tasks;
     using BenchmarkDotNet.Attributes;
     using BenchmarkDotNet.Jobs;
+    using Ignite.Sql;
     using Ignite.Table;
     using Tests;
 
     /// <summary>
     /// Measures SQL result set enumeration with two pages of data (two network requests per enumeration).
     /// <para />
-    /// Results on Intel Core i7-9700K, GC = Concurrent Server, Ubuntu 20.04:
-    /// |          Method |       Runtime |     Mean |   Error |   StdDev | Ratio |  Gen 0 |  Gen 1 | Allocated |
-    /// |---------------- |-------------- |---------:|--------:|---------:|------:|-------:|-------:|----------:|
-    /// |     ToListAsync |      .NET 6.0 | 218.4 us | 2.14 us |  2.01 us |  0.59 | 8.3008 | 1.7090 |    391 KB |
-    /// | AsyncEnumerable |      .NET 6.0 | 325.8 us | 6.14 us |  6.03 us |  0.88 | 8.7891 | 0.4883 |    392 KB |
-    /// |     ToListAsync | .NET Core 3.1 | 368.6 us | 7.14 us |  7.01 us |  1.00 | 7.3242 | 1.4648 |    383 KB |
-    /// | AsyncEnumerable | .NET Core 3.1 | 497.9 us | 9.59 us | 12.12 us |  1.35 | 7.8125 | 2.9297 |    384 KB |.
+    /// Results on Intel Core i7-7700HQ, .NET 6, GC = Concurrent Server, Ubuntu 20.04:
+    /// |                         Method |     Mean |     Error |    StdDev | Ratio | RatioSD |   Gen 0 | Allocated |
+    /// |------------------------------- |---------:|----------:|----------:|------:|--------:|--------:|----------:|
+    /// |                    ToListAsync | 2.325 ms | 0.0207 ms | 0.0183 ms |  1.00 |    0.00 | 15.6250 |    392 KB |
+    /// |                AsyncEnumerable | 2.487 ms | 0.0202 ms | 0.0189 ms |  1.07 |    0.01 | 15.6250 |    393 KB |
+    /// |                LinqToListAsync | 2.206 ms | 0.0121 ms | 0.0113 ms |  0.95 |    0.01 |       - |     80 KB |
+    /// | LinqSelectOneColumnToListAsync | 2.171 ms | 0.0278 ms | 0.0246 ms |  0.93 |    0.02 |       - |     57 KB |.
     /// </summary>
     [MemoryDiagnoser]
-    [SimpleJob(RuntimeMoniker.NetCoreApp31, baseline: true)]
-    /* [SimpleJob(RuntimeMoniker.Net60)] */
+    [SimpleJob(RuntimeMoniker.Net60)]
     public class ResultSetBenchmarks
     {
         private FakeServer? _server;
         private IIgniteClient? _client;
+        private IRecordView<Rec>? _recordView;
 
         [GlobalSetup]
         public async Task GlobalSetup()
         {
             _server = new FakeServer(disableOpsTracking: true);
             _client = await _server.ConnectClientAsync();
+
+            var table = await _client.Tables.GetTableAsync(FakeServer.ExistingTableName);
+            _recordView = table!.GetRecordView<Rec>();
         }
 
         [GlobalCleanup]
@@ -86,5 +91,33 @@ namespace Apache.Ignite.Benchmarks.Sql
                 throw new Exception("Wrong count");
             }
         }
+
+        [Benchmark]
+        public async Task LinqToListAsync()
+        {
+            await using var resultSet = await _recordView!.AsQueryable().ToResultSetAsync();
+
+            var rows = await resultSet.ToListAsync();
+
+            if (rows.Count != 1012)
+            {
+                throw new Exception("Wrong count");
+            }
+        }
+
+        [Benchmark]
+        public async Task LinqSelectOneColumnToListAsync()
+        {
+            await using var resultSet = await _recordView!.AsQueryable().Select(x => x.Id).ToResultSetAsync();
+
+            var rows = await resultSet.ToListAsync();
+
+            if (rows.Count != 1012)
+            {
+                throw new Exception("Wrong count");
+            }
+        }
+
+        private record Rec(int Id);
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Apache.Ignite.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Tests/Apache.Ignite.Tests.csproj
index e63e736904..fa4bed4b3f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/Apache.Ignite.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Apache.Ignite.Tests.csproj
@@ -27,8 +27,8 @@
 
     <ItemGroup>
         <PackageReference Include="Microsoft.NET.Test.Sdk" Version="16.9.4" />
-        <PackageReference Include="NUnit" Version="3.13.1" />
-        <PackageReference Include="NUnit3TestAdapter" Version="3.17.0" />
+        <PackageReference Include="NUnit" Version="3.13.3" />
+        <PackageReference Include="NUnit3TestAdapter" Version="4.3.1" />
         <PackageReference Include="System.Linq.Async" Version="6.0.1" />
     </ItemGroup>
 
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/IgniteTestsBase.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/IgniteTestsBase.cs
index 03712b777f..98dd7be17a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/IgniteTestsBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/IgniteTestsBase.cs
@@ -77,6 +77,8 @@ namespace Apache.Ignite.Tests
 
         protected IRecordView<PocoAllColumns> PocoAllColumnsView { get; private set; } = null!;
 
+        protected IRecordView<PocoAllColumnsNullable> PocoAllColumnsNullableView { get; private set; } = null!;
+
         [OneTimeSetUp]
         public async Task OneTimeSetUp()
         {
@@ -87,7 +89,10 @@ namespace Apache.Ignite.Tests
             Table = (await Client.Tables.GetTableAsync(TableName))!;
             TupleView = Table.RecordBinaryView;
             PocoView = Table.GetRecordView<Poco>();
-            PocoAllColumnsView = (await Client.Tables.GetTableAsync(TableAllColumnsName))!.GetRecordView<PocoAllColumns>();
+
+            var tableAllColumns = await Client.Tables.GetTableAsync(TableAllColumnsName);
+            PocoAllColumnsView = tableAllColumns!.GetRecordView<PocoAllColumns>();
+            PocoAllColumnsNullableView = tableAllColumns.GetRecordView<PocoAllColumnsNullable>();
         }
 
         [OneTimeTearDown]
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.Join.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.Join.cs
index 94e123a5f7..0f8b147aa1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.Join.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.Join.cs
@@ -220,7 +220,7 @@ public partial class LinqTests
     }
 
     [Test]
-    public void TestOuterJoin()
+    public void TestOuterJoinValueTypeKey()
     {
         var query1 = PocoIntView.AsQueryable(); // Sequential keys.
         var query2 = PocoShortView.AsQueryable(); // Sequential keys times 2.
@@ -257,6 +257,45 @@ public partial class LinqTests
             joinQuery.ToString());
     }
 
+    [Test]
+    public void TestOuterJoinReferenceTypeKey()
+    {
+        var query1 = PocoView.AsQueryable(); // Sequential values.
+        var query2 = PocoStringView.AsQueryable(); // Sequential values times 2.
+
+        var joinQuery = query1.Join(
+                inner: query2.DefaultIfEmpty(),
+                outerKeySelector: a => a.Val,
+                innerKeySelector: b => b.Val,
+                resultSelector: (a, b) => new
+                {
+                    Id = a.Key,
+                    Id2 = b.Key,
+                    Name = b.Val
+                })
+            .OrderBy(x => x.Id);
+
+        var res = joinQuery.ToList();
+
+        Assert.AreEqual(Count, res.Count);
+
+        Assert.AreEqual(1, res[1].Id);
+        Assert.AreEqual(null, res[1].Name);
+
+        Assert.AreEqual(2, res[2].Id);
+        Assert.AreEqual("v-2", res[2].Name);
+
+        Assert.AreEqual(3, res[3].Id);
+        Assert.AreEqual(null, res[3].Name);
+
+        StringAssert.Contains(
+            "select _T0.KEY, _T1.KEY, _T1.VAL " +
+            "from PUBLIC.TBL1 as _T0 " +
+            "left outer join (select * from PUBLIC.TBL_STRING as _T2 ) as _T1 on (_T1.VAL = _T0.VAL) " +
+            "order by (_T0.KEY) asc",
+            joinQuery.ToString());
+    }
+
     [Test]
     public void TestLocalCollectionJoinThrowsNotSupportedException()
     {
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.KvView.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.KvView.cs
index 140794627d..13deb43dcc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.KvView.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.KvView.cs
@@ -29,6 +29,8 @@ public partial class LinqTests
 {
     private IKeyValueView<KeyPoco, ValPoco> KvView { get; set; } = null!;
 
+    private IKeyValueView<KeyValPoco, KeyValPoco> KvView2 { get; set; } = null!;
+
     [Test]
     public void TestSelectPairKv()
     {
@@ -49,6 +51,30 @@ public partial class LinqTests
             query.ToString());
     }
 
+    [Test]
+    public void TestSelectPairIntersectingColumnsKv()
+    {
+        var query = KvView2.AsQueryable()
+            .Where(x => x.Key.Key > 3 && x.Value.Val != null)
+            .OrderBy(x => x.Key.Key);
+
+        // Key and value types share columns, all of them are populated.
+        List<KeyValuePair<KeyValPoco, KeyValPoco>> res = query.ToList();
+
+        Assert.AreEqual(4, res[0].Key.Key);
+        Assert.AreEqual(4, res[0].Value.Key);
+
+        Assert.AreEqual("v-4", res[0].Key.Val);
+        Assert.AreEqual("v-4", res[0].Value.Val);
+
+        StringAssert.Contains(
+            "select _T0.KEY, _T0.VAL " +
+            "from PUBLIC.TBL1 as _T0 " +
+            "where ((_T0.KEY > ?) and (_T0.VAL IS DISTINCT FROM ?)) " +
+            "order by (_T0.KEY) asc",
+            query.ToString());
+    }
+
     [Test]
     public void TestSelectKeyKv()
     {
@@ -144,14 +170,50 @@ public partial class LinqTests
             query.ToString());
     }
 
+    [Test]
+    public void TestGroupByAllAggregatesKv()
+    {
+        var query = KvView.AsQueryable()
+            .GroupBy(x => x.Key.Key)
+            .Select(x => new
+            {
+                x.Key,
+                Count = x.Count(),
+                Sum = x.Sum(a => a.Key.Key),
+                Avg = x.Average(a => a.Key.Key),
+                Min = x.Min(a => a.Key.Key),
+                Max = x.Max(a => a.Key.Key)
+            })
+            .OrderBy(x => x.Key);
+
+        var res = query.ToList();
+
+        Assert.AreEqual(2, res[2].Key);
+        Assert.AreEqual(1, res[2].Count);
+        Assert.AreEqual(2, res[2].Sum);
+        Assert.AreEqual(2, res[2].Avg);
+        Assert.AreEqual(2, res[2].Min);
+        Assert.AreEqual(2, res[2].Max);
+
+        StringAssert.Contains(
+            "select _T0.KEY, count(*) as COUNT, sum(_T0.KEY) as SUM, avg(_T0.KEY) as AVG, min(_T0.KEY) as MIN, max(_T0.KEY) as MAX " +
+            "from PUBLIC.TBL1 as _T0 " +
+            "group by (_T0.KEY) " +
+            "order by (_T0.KEY) asc",
+            query.ToString());
+    }
+
     [OneTimeSetUp]
     protected void InitKvView()
     {
         KvView = Table.GetKeyValueView<KeyPoco, ValPoco>();
+        KvView2 = Table.GetKeyValueView<KeyValPoco, KeyValPoco>();
     }
 
     // ReSharper disable ClassNeverInstantiated.Local
     private record KeyPoco(long Key);
 
     private record ValPoco(string? Val);
+
+    private record KeyValPoco(long Key, string? Val);
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.cs
index 8176c3995f..cb7c2ab09b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/LinqTests.cs
@@ -50,6 +50,8 @@ public partial class LinqTests : IgniteTestsBase
 
     private IRecordView<PocoDecimal> PocoDecimalView { get; set; } = null!;
 
+    private IRecordView<PocoString> PocoStringView { get; set; } = null!;
+
     [OneTimeSetUp]
     public async Task InsertData()
     {
@@ -60,6 +62,7 @@ public partial class LinqTests : IgniteTestsBase
         PocoFloatView = (await Client.Tables.GetTableAsync(TableFloatName))!.GetRecordView<PocoFloat>();
         PocoDoubleView = (await Client.Tables.GetTableAsync(TableDoubleName))!.GetRecordView<PocoDouble>();
         PocoDecimalView = (await Client.Tables.GetTableAsync(TableDecimalName))!.GetRecordView<PocoDecimal>();
+        PocoStringView = (await Client.Tables.GetTableAsync(TableStringName))!.GetRecordView<PocoString>();
 
         for (int i = 0; i < Count; i++)
         {
@@ -73,6 +76,8 @@ public partial class LinqTests : IgniteTestsBase
             await PocoFloatView.UpsertAsync(null, new(i, i));
             await PocoDoubleView.UpsertAsync(null, new(i, i));
             await PocoDecimalView.UpsertAsync(null, new(i, i));
+
+            await PocoStringView.UpsertAsync(null, new("k-" + i, "v-" + i * 2));
         }
     }
 
@@ -471,6 +476,31 @@ public partial class LinqTests : IgniteTestsBase
         Assert.AreEqual(expected, query.ToString());
     }
 
+    [Test]
+    public void TestSelectDecimalIntoAnonymousTypeUsesCorrectScale()
+    {
+        var query = PocoDecimalView.AsQueryable()
+            .OrderByDescending(x => x.Val)
+            .Select(x => new
+            {
+                Id = x.Key
+            });
+
+        var res = query.ToList();
+        Assert.AreEqual(9.0m, res[0].Id);
+    }
+
+    [Test]
+    public void TestSelectDecimalIntoUserDefinedTypeUsesCorrectScale()
+    {
+        var query = PocoDecimalView.AsQueryable()
+            .OrderByDescending(x => x.Val)
+            .Select(x => new PocoDecimal(x.Val, x.Key));
+
+        var res = query.ToList();
+        Assert.AreEqual(9.0m, res[0].Val);
+    }
+
     private record PocoByte(sbyte Key, sbyte Val);
 
     private record PocoShort(short Key, short Val);
@@ -484,4 +514,6 @@ public partial class LinqTests : IgniteTestsBase
     private record PocoDouble(double Key, double Val);
 
     private record PocoDecimal(decimal Key, decimal Val);
+
+    private record PocoString(string Key, string Val);
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/ResultSelectorCacheKeyTests.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/ResultSelectorCacheKeyTests.cs
new file mode 100644
index 0000000000..030bb17415
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Linq/ResultSelectorCacheKeyTests.cs
@@ -0,0 +1,131 @@
+/*
+ * 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.Ignite.Tests.Linq;
+
+using System.Linq;
+using Ignite.Sql;
+using Internal.Linq;
+using Internal.Sql;
+using NUnit.Framework;
+
+/// <summary>
+/// Tests for <see cref="ResultSelectorCacheKey{T}"/>.
+/// </summary>
+public class ResultSelectorCacheKeyTests
+{
+    [Test]
+    public void TestKeysWithSameColumnsAreEqual()
+    {
+        var target = new object();
+        var columns = GetColumns();
+
+        var key1 = new ResultSelectorCacheKey<object>(target, columns, defaultIfNull: false);
+        var key2 = new ResultSelectorCacheKey<object>(target, columns, defaultIfNull: false);
+
+        Assert.AreEqual(key1, key2);
+        Assert.AreEqual(key1.GetHashCode(), key2.GetHashCode());
+        Assert.IsTrue(key1 == key2);
+    }
+
+    [Test]
+    public void TestKeysWithSameColumnTypeAndScaleAreEqual()
+    {
+        var target = new object();
+        var columns = GetColumns();
+        var columns2 = GetColumns();
+
+        // Change everything except type and scale.
+        columns2[0] = columns2[0] with { Name = "foo", Nullable = false, Origin = new ColumnOrigin("a", "b", "c"), Precision = 123 };
+
+        var key1 = new ResultSelectorCacheKey<object>(target, columns, defaultIfNull: false);
+        var key2 = new ResultSelectorCacheKey<object>(target, columns2, defaultIfNull: false);
+
+        Assert.AreEqual(key1, key2);
+        Assert.AreEqual(key1.GetHashCode(), key2.GetHashCode());
+        Assert.IsTrue(key1 == key2);
+    }
+
+    [Test]
+    public void TestKeysWithDifferentColumnTypesAreNotEqual()
+    {
+        var target = new object();
+        var columns = GetColumns();
+        var columns2 = GetColumns();
+
+        columns2[0] = columns2[0] with { Type = SqlColumnType.String };
+
+        var key1 = new ResultSelectorCacheKey<object>(target, columns, defaultIfNull: false);
+        var key2 = new ResultSelectorCacheKey<object>(target, columns2, defaultIfNull: false);
+
+        Assert.AreNotEqual(key1, key2);
+        Assert.AreNotEqual(key1.GetHashCode(), key2.GetHashCode());
+        Assert.IsFalse(key1 == key2);
+    }
+
+    [Test]
+    public void TestKeysWithDifferentColumnScalesAreNotEqual()
+    {
+        var target = new object();
+        var columns = GetColumns();
+        var columns2 = GetColumns();
+
+        columns2[0] = columns2[0] with { Scale = -1 };
+
+        var key1 = new ResultSelectorCacheKey<object>(target, columns, defaultIfNull: false);
+        var key2 = new ResultSelectorCacheKey<object>(target, columns2, defaultIfNull: false);
+
+        Assert.AreNotEqual(key1, key2);
+        Assert.AreNotEqual(key1.GetHashCode(), key2.GetHashCode());
+        Assert.IsFalse(key1 == key2);
+    }
+
+    [Test]
+    public void TestKeysWithSameColumnsDifferentOrderAreNotEqual()
+    {
+        var target = new object();
+        var columns = GetColumns();
+
+        var key1 = new ResultSelectorCacheKey<object>(target, columns, defaultIfNull: false);
+        var key2 = new ResultSelectorCacheKey<object>(target, columns.Reverse().ToList(), defaultIfNull: false);
+
+        Assert.AreNotEqual(key1, key2);
+        Assert.AreNotEqual(key1.GetHashCode(), key2.GetHashCode());
+        Assert.IsFalse(key1 == key2);
+    }
+
+    private static ColumnMetadata[] GetColumns()
+    {
+        return new[]
+        {
+            new ColumnMetadata(
+                Name: "c1",
+                Type: SqlColumnType.Date,
+                Precision: 1,
+                Scale: 2,
+                Nullable: true,
+                Origin: null),
+            new ColumnMetadata(
+                Name: "c2",
+                Type: SqlColumnType.Float,
+                Precision: 4,
+                Scale: 6,
+                Nullable: false,
+                Origin: null)
+        };
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Table/PocoAllColumnsNullable.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/Table/PocoAllColumnsNullable.cs
new file mode 100644
index 0000000000..df7d4cb4ff
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Table/PocoAllColumnsNullable.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.
+ */
+
+namespace Apache.Ignite.Tests.Table
+{
+    using System;
+    using System.Collections;
+    using System.Diagnostics.CodeAnalysis;
+    using NodaTime;
+
+    /// <summary>
+    /// Test user object.
+    /// </summary>
+    [SuppressMessage("Microsoft.Naming", "CA1720:AvoidTypeNamesInParameters", Justification = "POCO mapping.")]
+    [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly", Justification = "POCO mapping.")]
+    [SuppressMessage("Microsoft.Performance", "CA1819:PropertiesShouldNotReturnArrays", Justification = "POCO mapping.")]
+    public record PocoAllColumnsNullable(
+        long Key,
+        string? Str = null,
+        sbyte? Int8 = null,
+        short? Int16 = null,
+        int? Int32 = null,
+        long? Int64 = null,
+        float? Float = null,
+        double? Double = null,
+        Guid? Uuid = null,
+        LocalDate? Date = null,
+        BitArray? BitMask = null,
+        LocalTime? Time = null,
+        LocalDateTime? DateTime = null,
+        Instant? Timestamp = null,
+        byte[]? Blob = null,
+        decimal? Decimal = null);
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Table/RecordViewPocoTests.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/Table/RecordViewPocoTests.cs
index d608831528..6a7ff586d1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/Table/RecordViewPocoTests.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Table/RecordViewPocoTests.cs
@@ -649,6 +649,34 @@ namespace Apache.Ignite.Tests.Table
             Assert.AreEqual(poco.DateTime, res.DateTime);
         }
 
+        [Test]
+        [Ignore("IGNITE-18329 Add support for nullable value type mapping")]
+        public async Task TestAllColumnsPocoNullable()
+        {
+            var pocoView = PocoAllColumnsNullableView;
+
+            var poco = new PocoAllColumnsNullable(123);
+            await pocoView.UpsertAsync(null, poco);
+
+            var res = (await pocoView.GetAsync(null, poco)).Value;
+
+            Assert.AreEqual(poco.Blob, res.Blob);
+            Assert.AreEqual(poco.Date, res.Date);
+            Assert.AreEqual(poco.Decimal, res.Decimal);
+            Assert.AreEqual(poco.Double, res.Double);
+            Assert.AreEqual(poco.Float, res.Float);
+            Assert.AreEqual(poco.Int8, res.Int8);
+            Assert.AreEqual(poco.Int16, res.Int16);
+            Assert.AreEqual(poco.Int32, res.Int32);
+            Assert.AreEqual(poco.Int64, res.Int64);
+            Assert.AreEqual(poco.Str, res.Str);
+            Assert.AreEqual(poco.Uuid, res.Uuid);
+            Assert.AreEqual(poco.BitMask, res.BitMask);
+            Assert.AreEqual(poco.Timestamp, res.Timestamp);
+            Assert.AreEqual(poco.Time, res.Time);
+            Assert.AreEqual(poco.DateTime, res.DateTime);
+        }
+
         [Test]
         public async Task TestUnsupportedColumnTypeThrowsException()
         {
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
index 9994010d45..be6c5c936f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
+++ b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
@@ -18,6 +18,7 @@
 <wpf:ResourceDictionary xml:space="preserve" xmlns:x="http://schemas.microsoft.com/winfx/2006/xaml" xmlns:s="clr-namespace:System;assembly=mscorlib" xmlns:ss="urn:shemas-jetbrains-com:settings-storage-xaml" xmlns:wpf="http://schemas.microsoft.com/winfx/2006/xaml/presentation">
 	<s:Int64 x:Key="/Default/CodeStyle/CodeFormatting/CSharpFormat/WRAP_LIMIT/@EntryValue">140</s:Int64>
 	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/AddImportsToDeepestScope/@EntryValue">True</s:Boolean>
+	<s:String x:Key="/Default/CodeStyle/Naming/CSharpNaming/Abbreviations/=IL/@EntryIndexedValue">IL</s:String>
 	<s:Boolean x:Key="/Default/Environment/SettingsMigration/IsMigratorApplied/=JetBrains_002EReSharper_002EPsi_002ECSharp_002ECodeStyle_002ECSharpKeepExistingMigration/@EntryIndexedValue">True</s:Boolean>
 	<s:Boolean x:Key="/Default/Environment/SettingsMigration/IsMigratorApplied/=JetBrains_002EReSharper_002EPsi_002ECSharp_002ECodeStyle_002ECSharpPlaceEmbeddedOnSameLineMigration/@EntryIndexedValue">True</s:Boolean>
 	<s:Boolean x:Key="/Default/Environment/SettingsMigration/IsMigratorApplied/=JetBrains_002EReSharper_002EPsi_002ECSharp_002ECodeStyle_002ECSharpUseContinuousIndentInsideBracesMigration/@EntryIndexedValue">True</s:Boolean>
diff --git a/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj b/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
index 81dcd23494..e24afcff84 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
@@ -28,8 +28,8 @@
   </PropertyGroup>
 
   <ItemGroup>
-    <PackageReference Include="JetBrains.Annotations" Version="2022.1.0" PrivateAssets="all" />
-    <PackageReference Include="MessagePack" Version="[2.1.80,)" />
+    <PackageReference Include="JetBrains.Annotations" Version="2022.3.1" PrivateAssets="all" />
+    <PackageReference Include="MessagePack" Version="[2.1.90,)" />
     <PackageReference Include="NodaTime" Version="[3.*,)" />
     <PackageReference Include="Remotion.Linq" Version="2.2.0" />
   </ItemGroup>
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/IgniteQueryExecutor.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/IgniteQueryExecutor.cs
index fd049619ff..2b977174b0 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/IgniteQueryExecutor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/IgniteQueryExecutor.cs
@@ -20,18 +20,11 @@ namespace Apache.Ignite.Internal.Linq;
 using System;
 using System.Collections.Generic;
 using System.Diagnostics.CodeAnalysis;
-using System.Globalization;
-using System.Linq.Expressions;
-using System.Runtime.Serialization;
 using System.Threading.Tasks;
 using Ignite.Sql;
 using Ignite.Transactions;
-using Proto.BinaryTuple;
 using Remotion.Linq;
-using Remotion.Linq.Clauses;
-using Remotion.Linq.Clauses.Expressions;
 using Sql;
-using Table.Serialization;
 
 /// <summary>
 /// Fields query executor.
@@ -121,105 +114,13 @@ internal sealed class IgniteQueryExecutor : IQueryExecutor
         IResultSet<T> resultSet = await _sql.ExecuteAsyncInternal(
             _transaction,
             statement,
-            cols => GetResultSelector<T>(cols, queryModel.SelectClause.Selector),
+            cols => ResultSelector.Get<T>(cols, queryModel.SelectClause.Selector, defaultIfNull: qryData.HasOuterJoins),
             qryData.Parameters)
             .ConfigureAwait(false);
 
         return resultSet;
     }
 
-    /// <summary>
-    /// Gets the result selector.
-    /// </summary>
-    private static RowReader<T> GetResultSelector<T>(IReadOnlyList<IColumnMetadata> columns, Expression selectorExpression)
-    {
-        // TODO: IGNITE-18136 Replace reflection with emitted delegates.
-        if (selectorExpression is NewExpression newExpr)
-        {
-            return (IReadOnlyList<IColumnMetadata> cols, ref BinaryTupleReader reader) =>
-            {
-                var args = new object?[cols.Count];
-
-                for (int i = 0; i < cols.Count; i++)
-                {
-                    var val = Sql.ReadColumnValue(ref reader, cols[i], i);
-
-                    if (val != null)
-                    {
-                        val = Convert.ChangeType(val, newExpr.Arguments[i].Type, CultureInfo.InvariantCulture);
-                    }
-
-                    args[i] = val;
-                }
-
-                return (T)newExpr.Constructor!.Invoke(args);
-            };
-        }
-
-        if (columns.Count == 1 && typeof(T).ToSqlColumnType() is not null)
-        {
-            return (IReadOnlyList<IColumnMetadata> cols, ref BinaryTupleReader reader) =>
-                (T)Convert.ChangeType(Sql.ReadColumnValue(ref reader, cols[0], 0)!, typeof(T), CultureInfo.InvariantCulture);
-        }
-
-        if (typeof(T).GetKeyValuePairTypes() is var (keyType, valType))
-        {
-            return (IReadOnlyList<IColumnMetadata> cols, ref BinaryTupleReader reader) =>
-            {
-                var key = FormatterServices.GetUninitializedObject(keyType);
-                var val = FormatterServices.GetUninitializedObject(valType);
-
-                for (int i = 0; i < cols.Count; i++)
-                {
-                    var col = cols[i];
-                    var colVal = Sql.ReadColumnValue(ref reader, col, i);
-
-                    SetColumnValue(col, colVal, key, keyType);
-                    SetColumnValue(col, colVal, val, valType);
-                }
-
-                return (T)Activator.CreateInstance(typeof(T), key, val)!;
-            };
-        }
-
-        if (selectorExpression is QuerySourceReferenceExpression
-            {
-                ReferencedQuerySource: IFromClause { FromExpression: SubQueryExpression subQuery }
-            })
-        {
-            // Select everything from a sub-query - use nested selector.
-            return GetResultSelector<T>(columns, subQuery.QueryModel.SelectClause.Selector);
-        }
-
-        return (IReadOnlyList<IColumnMetadata> cols, ref BinaryTupleReader reader) =>
-        {
-            var res = (T)FormatterServices.GetUninitializedObject(typeof(T));
-
-            for (int i = 0; i < cols.Count; i++)
-            {
-                var col = cols[i];
-                var val = Sql.ReadColumnValue(ref reader, col, i);
-
-                SetColumnValue(col, val, res, typeof(T));
-            }
-
-            return res;
-        };
-    }
-
-    private static void SetColumnValue<T>(IColumnMetadata col, object? val, T res, Type type)
-    {
-        if (type.GetFieldByColumnName(col.Name) is {} field)
-        {
-            if (val != null)
-            {
-                val = Convert.ChangeType(val, field.FieldType, CultureInfo.InvariantCulture);
-            }
-
-            field.SetValue(res, val);
-        }
-    }
-
     [SuppressMessage("Reliability", "CA2007:Consider calling ConfigureAwait on the awaited task", Justification = "False positive.")]
     private async Task<T?> ExecuteSingleInternalAsync<T>(QueryModel queryModel, bool returnDefaultWhenEmpty)
     {
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/IgniteQueryModelVisitor.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/IgniteQueryModelVisitor.cs
index c685d0bacc..0758fbf76e 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/IgniteQueryModelVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/IgniteQueryModelVisitor.cs
@@ -46,6 +46,9 @@ internal sealed class IgniteQueryModelVisitor : QueryModelVisitorBase
     /** */
     private readonly AliasDictionary _aliases = new();
 
+    /** */
+    private bool _hasOuterJoins;
+
     /// <summary>
     /// Gets the builder.
     /// </summary>
@@ -72,7 +75,7 @@ internal sealed class IgniteQueryModelVisitor : QueryModelVisitorBase
 
         var qryText = _builder.TrimEnd().ToString();
 
-        return new QueryData(qryText, _parameters);
+        return new QueryData(qryText, _parameters, _hasOuterJoins);
     }
 
     /** <inheritdoc /> */
@@ -125,11 +128,10 @@ internal sealed class IgniteQueryModelVisitor : QueryModelVisitorBase
 
         if (queryable != null)
         {
-            var subQuery = joinClause.InnerSequence as SubQueryExpression;
-
-            if (subQuery != null)
+            if (joinClause.InnerSequence is SubQueryExpression subQuery)
             {
                 var isOuter = subQuery.QueryModel.ResultOperators.OfType<DefaultIfEmptyResultOperator>().Any();
+                _hasOuterJoins |= isOuter;
 
                 _builder.AppendFormat(CultureInfo.InvariantCulture, "{0} join (", isOuter ? "left outer" : "inner");
 
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/QueryData.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/QueryData.cs
index 34fa8eb9ee..b880efbedb 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/QueryData.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/QueryData.cs
@@ -22,4 +22,4 @@ using System.Collections.Generic;
 /// <summary>
 /// Query data holder.
 /// </summary>
-internal sealed record QueryData(string QueryText, ICollection<object?> Parameters);
+internal sealed record QueryData(string QueryText, ICollection<object?> Parameters, bool HasOuterJoins);
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/ResultSelector.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/ResultSelector.cs
new file mode 100644
index 0000000000..65b3e05460
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/ResultSelector.cs
@@ -0,0 +1,305 @@
+/*
+ * 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.Ignite.Internal.Linq;
+
+using System;
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+using System.Linq;
+using System.Linq.Expressions;
+using System.Reflection;
+using System.Reflection.Emit;
+using System.Threading;
+using Ignite.Sql;
+using Proto.BinaryTuple;
+using Remotion.Linq.Clauses;
+using Remotion.Linq.Clauses.Expressions;
+using Sql;
+using Table.Serialization;
+
+/// <summary>
+/// Result selector cache.
+/// </summary>
+internal static class ResultSelector
+{
+    private static readonly ConcurrentDictionary<ResultSelectorCacheKey<ConstructorInfo>, object> CtorCache = new();
+
+    private static readonly ConcurrentDictionary<ResultSelectorCacheKey<Type>, object> SingleColumnReaderCache = new();
+
+    private static readonly ConcurrentDictionary<ResultSelectorCacheKey<Type>, object> ReaderCache = new();
+
+    private static readonly ConcurrentDictionary<ResultSelectorCacheKey<(Type Key, Type Val)>, object> KvReaderCache = new();
+
+    private static long _idCounter;
+
+    /// <summary>
+    /// Gets the result selector.
+    /// <para />
+    /// Some of the logic here is similar to <see cref="ObjectSerializerHandler{T}"/>, but there are subtle differences:
+    /// SQL has a different set of types;
+    /// LINQ handles type conversion when possible;
+    /// LINQ allows more ways to instantiate resulting objects.
+    /// </summary>
+    /// <param name="columns">Columns.</param>
+    /// <param name="selectorExpression">Selector expression.</param>
+    /// <param name="defaultIfNull">Whether to read null values as default for value types
+    /// (when <see cref="Queryable.DefaultIfEmpty{TSource}(System.Linq.IQueryable{TSource})"/> is used).</param>
+    /// <typeparam name="T">Result type.</typeparam>
+    /// <returns>Row reader.</returns>
+    public static RowReader<T> Get<T>(IReadOnlyList<IColumnMetadata> columns, Expression selectorExpression, bool defaultIfNull)
+    {
+        // Anonymous type projections use a constructor call. But user-defined types can also be used with constructor call.
+        if (selectorExpression is NewExpression newExpr)
+        {
+            var ctorInfo = newExpr.Constructor!;
+            var ctorCacheKey = new ResultSelectorCacheKey<ConstructorInfo>(ctorInfo, columns, defaultIfNull);
+
+            return (RowReader<T>)CtorCache.GetOrAdd(
+                ctorCacheKey,
+                static k => EmitConstructorReader<T>(k.Target, k.Columns, k.DefaultIfNull));
+        }
+
+        if (columns.Count == 1 && typeof(T).ToSqlColumnType() is not null)
+        {
+            var singleColumnCacheKey = new ResultSelectorCacheKey<Type>(typeof(T), columns, defaultIfNull);
+
+            return (RowReader<T>)SingleColumnReaderCache.GetOrAdd(
+                singleColumnCacheKey,
+                static k => EmitSingleColumnReader<T>(k.Columns[0], k.DefaultIfNull));
+        }
+
+        if (typeof(T).GetKeyValuePairTypes() is var (keyType, valType))
+        {
+            var kvCacheKey = new ResultSelectorCacheKey<(Type Key, Type Val)>((keyType, valType), columns, defaultIfNull);
+
+            return (RowReader<T>)KvReaderCache.GetOrAdd(
+                kvCacheKey,
+                static k => EmitKvPairReader<T>(k.Columns, k.Target.Key, k.Target.Val, k.DefaultIfNull));
+        }
+
+        if (selectorExpression is QuerySourceReferenceExpression
+            {
+                ReferencedQuerySource: IFromClause { FromExpression: SubQueryExpression subQuery }
+            })
+        {
+            // Select everything from a sub-query - use nested selector.
+            return Get<T>(columns, subQuery.QueryModel.SelectClause.Selector, defaultIfNull);
+        }
+
+        var readerCacheKey = new ResultSelectorCacheKey<Type>(typeof(T), columns, defaultIfNull);
+
+        return (RowReader<T>)ReaderCache.GetOrAdd(
+            readerCacheKey,
+            static k => EmitUninitializedObjectReader<T>(k.Columns, k.DefaultIfNull));
+    }
+
+    private static RowReader<T> EmitSingleColumnReader<T>(IColumnMetadata column, bool defaultIfNull)
+    {
+        var method = new DynamicMethod(
+            name: $"SingleColumnFromBinaryTupleReader_{typeof(T).FullName}_{GetNextId()}",
+            returnType: typeof(T),
+            parameterTypes: new[] { typeof(IReadOnlyList<IColumnMetadata>), typeof(BinaryTupleReader).MakeByRefType() },
+            m: typeof(IIgnite).Module,
+            skipVisibility: true);
+
+        var il = method.GetILGenerator();
+
+        EmitReadToStack(il, column, typeof(T), 0, defaultIfNull);
+        il.Emit(OpCodes.Ret);
+
+        return (RowReader<T>)method.CreateDelegate(typeof(RowReader<T>));
+    }
+
+    private static RowReader<T> EmitConstructorReader<T>(
+        ConstructorInfo ctorInfo,
+        IReadOnlyList<IColumnMetadata> columns,
+        bool defaultAsNull)
+    {
+        var method = new DynamicMethod(
+            name: $"ConstructorFromBinaryTupleReader_{typeof(T).FullName}_{GetNextId()}",
+            returnType: typeof(T),
+            parameterTypes: new[] { typeof(IReadOnlyList<IColumnMetadata>), typeof(BinaryTupleReader).MakeByRefType() },
+            m: typeof(IIgnite).Module,
+            skipVisibility: true);
+
+        var il = method.GetILGenerator();
+        var ctorParams = ctorInfo.GetParameters();
+
+        if (ctorParams.Length != columns.Count)
+        {
+            throw new InvalidOperationException("Constructor parameter count does not match column count, can't emit row reader.");
+        }
+
+        // Read all constructor parameters and push them to the evaluation stack.
+        for (var index = 0; index < ctorParams.Length; index++)
+        {
+            var paramType = ctorParams[index].ParameterType;
+            EmitReadToStack(il, columns[index], paramType, index, defaultAsNull);
+        }
+
+        il.Emit(OpCodes.Newobj, ctorInfo);
+        il.Emit(OpCodes.Ret);
+
+        return (RowReader<T>)method.CreateDelegate(typeof(RowReader<T>));
+    }
+
+    private static RowReader<T> EmitUninitializedObjectReader<T>(
+        IReadOnlyList<IColumnMetadata> columns,
+        bool defaultAsNull)
+    {
+        var method = new DynamicMethod(
+            name: $"UninitializedObjectFromBinaryTupleReader_{typeof(T).FullName}_{GetNextId()}",
+            returnType: typeof(T),
+            parameterTypes: new[] { typeof(IReadOnlyList<IColumnMetadata>), typeof(BinaryTupleReader).MakeByRefType() },
+            m: typeof(IIgnite).Module,
+            skipVisibility: true);
+
+        var il = method.GetILGenerator();
+
+        var resObj = il.DeclareAndInitLocal(typeof(T));
+
+        for (var index = 0; index < columns.Count; index++)
+        {
+            var col = columns[index];
+
+            EmitFieldRead(il, resObj, col, index, defaultAsNull);
+        }
+
+        il.Emit(OpCodes.Ldloc_0); // res
+        il.Emit(OpCodes.Ret);
+
+        return (RowReader<T>)method.CreateDelegate(typeof(RowReader<T>));
+    }
+
+    private static RowReader<T> EmitKvPairReader<T>(
+        IReadOnlyList<IColumnMetadata> columns,
+        Type keyType,
+        Type valType,
+        bool defaultAsNull)
+    {
+        var method = new DynamicMethod(
+            name: $"KvPairFromBinaryTupleReader_{typeof(T).FullName}_{GetNextId()}",
+            returnType: typeof(T),
+            parameterTypes: new[] { typeof(IReadOnlyList<IColumnMetadata>), typeof(BinaryTupleReader).MakeByRefType() },
+            m: typeof(IIgnite).Module,
+            skipVisibility: true);
+
+        var il = method.GetILGenerator();
+
+        var key = il.DeclareAndInitLocal(keyType);
+        var val = il.DeclareAndInitLocal(valType);
+
+        for (var index = 0; index < columns.Count; index++)
+        {
+            var col = columns[index];
+
+            EmitFieldRead(il, key, col, index, defaultAsNull);
+            EmitFieldRead(il, val, col, index, defaultAsNull);
+        }
+
+        il.Emit(OpCodes.Ldloc_0); // key
+        il.Emit(OpCodes.Ldloc_1); // val
+
+        il.Emit(OpCodes.Newobj, typeof(T).GetConstructor(new[] { keyType, valType })!);
+        il.Emit(OpCodes.Ret);
+
+        return (RowReader<T>)method.CreateDelegate(typeof(RowReader<T>));
+    }
+
+    private static void EmitReadToStack(ILGenerator il, IColumnMetadata col, Type targetType, int index, bool defaultAsNull)
+    {
+        Label endParamLabel = il.DefineLabel();
+
+        if (defaultAsNull)
+        {
+            // if (reader.IsNull(index)) return default;
+            Label notNullLabel = il.DefineLabel();
+            il.Emit(OpCodes.Ldarg_1); // Reader.
+            il.Emit(OpCodes.Ldc_I4, index); // Index.
+            il.Emit(OpCodes.Call, BinaryTupleMethods.IsNull);
+            il.Emit(OpCodes.Brfalse_S, notNullLabel);
+
+            if (targetType.IsValueType)
+            {
+                var local = il.DeclareLocal(targetType);
+                il.Emit(OpCodes.Ldloca_S, local);
+                il.Emit(OpCodes.Initobj, targetType); // Load default value into local.
+                il.Emit(OpCodes.Ldloc, local); // Load local value onto stack for constructor call.
+            }
+            else
+            {
+                il.Emit(OpCodes.Ldnull);
+            }
+
+            il.Emit(OpCodes.Br_S, endParamLabel);
+            il.MarkLabel(notNullLabel);
+        }
+
+        il.Emit(OpCodes.Ldarg_1); // Reader.
+        il.Emit(OpCodes.Ldc_I4, index); // Index.
+
+        if (col.Type == SqlColumnType.Decimal)
+        {
+            il.Emit(OpCodes.Ldc_I4, col.Scale);
+        }
+
+        var colType = col.Type.ToClrType();
+        il.Emit(OpCodes.Call, BinaryTupleMethods.GetReadMethod(colType));
+
+        il.EmitConv(colType, targetType);
+        il.MarkLabel(endParamLabel);
+    }
+
+    private static void EmitFieldRead(ILGenerator il, LocalBuilder targetObj, IColumnMetadata col, int colIndex, bool defaultAsNull)
+    {
+        if (targetObj.LocalType.GetFieldByColumnName(col.Name) is not { } field)
+        {
+            return;
+        }
+
+        Label endFieldLabel = il.DefineLabel();
+
+        if (defaultAsNull)
+        {
+            // if (reader.IsNull(index)) continue;
+            il.Emit(OpCodes.Ldarg_1); // Reader.
+            il.Emit(OpCodes.Ldc_I4, colIndex); // Index.
+            il.Emit(OpCodes.Call, BinaryTupleMethods.IsNull);
+            il.Emit(OpCodes.Brtrue_S, endFieldLabel);
+        }
+
+        il.Emit(targetObj.LocalType.IsValueType ? OpCodes.Ldloca_S : OpCodes.Ldloc, targetObj); // res
+        il.Emit(OpCodes.Ldarg_1); // Reader.
+        il.Emit(OpCodes.Ldc_I4, colIndex); // Index.
+
+        if (col.Type == SqlColumnType.Decimal)
+        {
+            il.Emit(OpCodes.Ldc_I4, col.Scale);
+        }
+
+        var colType = col.Type.ToClrType();
+        il.Emit(OpCodes.Call, BinaryTupleMethods.GetReadMethod(colType));
+
+        il.EmitConv(colType, field.FieldType);
+        il.Emit(OpCodes.Stfld, field); // res.field = value
+
+        il.MarkLabel(endFieldLabel);
+    }
+
+    private static long GetNextId() => Interlocked.Increment(ref _idCounter);
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/ResultSelectorCacheKey.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/ResultSelectorCacheKey.cs
new file mode 100644
index 0000000000..481c3810d0
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Linq/ResultSelectorCacheKey.cs
@@ -0,0 +1,123 @@
+/*
+ * 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.Ignite.Internal.Linq;
+
+using System;
+using System.Collections.Generic;
+using Ignite.Sql;
+
+/// <summary>
+/// Key for <see cref="ResultSelector"/> cached delegates. Equality logic is based on column types.
+/// </summary>
+/// <typeparam name="T">Target type.</typeparam>
+internal readonly struct ResultSelectorCacheKey<T> : IEquatable<ResultSelectorCacheKey<T>>
+    where T : notnull
+{
+    /// <summary>
+    /// Initializes a new instance of the <see cref="ResultSelectorCacheKey{T}"/> struct.
+    /// </summary>
+    /// <param name="target">Target object (can be type, constructor, etc).</param>
+    /// <param name="columns">Columns.</param>
+    /// <param name="defaultIfNull">Whether to read null values as default for value types.</param>
+    public ResultSelectorCacheKey(T target, IReadOnlyList<IColumnMetadata> columns, bool defaultIfNull)
+    {
+        Target = target;
+        Columns = columns;
+        DefaultIfNull = defaultIfNull;
+    }
+
+    /// <summary>
+    /// Gets columns.
+    /// </summary>
+    public IReadOnlyList<IColumnMetadata> Columns { get; }
+
+    /// <summary>
+    /// Gets a value indicating whether null values should be interpreted as default for value types.
+    /// </summary>
+    public bool DefaultIfNull { get; }
+
+    /// <summary>
+    /// Gets target object (can be type, constructor, etc).
+    /// </summary>
+    public T Target { get; }
+
+    public static bool operator ==(ResultSelectorCacheKey<T> left, ResultSelectorCacheKey<T> right)
+    {
+        return left.Equals(right);
+    }
+
+    public static bool operator !=(ResultSelectorCacheKey<T> left, ResultSelectorCacheKey<T> right)
+    {
+        return !(left == right);
+    }
+
+    /// <inheritdoc/>
+    public bool Equals(ResultSelectorCacheKey<T> other)
+    {
+        if (!Target.Equals(other.Target))
+        {
+            return false;
+        }
+
+        if (DefaultIfNull != other.DefaultIfNull)
+        {
+            return false;
+        }
+
+        if (Columns.Count != other.Columns.Count)
+        {
+            return false;
+        }
+
+        for (var i = 0; i < Columns.Count; i++)
+        {
+            if (Columns[i].Type != other.Columns[i].Type)
+            {
+                return false;
+            }
+
+            if (Columns[i].Scale != other.Columns[i].Scale)
+            {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    /// <inheritdoc/>
+    public override bool Equals(object? obj) =>
+        obj is ResultSelectorCacheKey<T> other && Equals(other);
+
+    /// <inheritdoc/>
+    public override int GetHashCode()
+    {
+        HashCode hash = default;
+
+        hash.Add(Target);
+        hash.Add(DefaultIfNull);
+
+        foreach (var column in Columns)
+        {
+            hash.Add(column.Type);
+            hash.Add(column.Scale);
+        }
+
+        return hash.ToHashCode();
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Proto/BinaryTuple/BinaryTupleReader.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Proto/BinaryTuple/BinaryTupleReader.cs
index 5a9b228a0a..53b54392dc 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Proto/BinaryTuple/BinaryTupleReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Proto/BinaryTuple/BinaryTupleReader.cs
@@ -102,6 +102,13 @@ namespace Apache.Ignite.Internal.Proto.BinaryTuple
             var s => unchecked((sbyte)s[0])
         };
 
+        /// <summary>
+        /// Gets a byte value as bool.
+        /// </summary>
+        /// <param name="index">Index.</param>
+        /// <returns>Value.</returns>
+        public bool GetByteAsBool(int index) => GetByte(index) != 0;
+
         /// <summary>
         /// Gets a short value.
         /// </summary>
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Sql/Sql.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Sql/Sql.cs
index 1221ce9c53..4579feb28d 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Sql/Sql.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Sql/Sql.cs
@@ -78,7 +78,7 @@ namespace Apache.Ignite.Internal.Sql
 
             return col.Type switch
             {
-                SqlColumnType.Boolean => reader.GetByte(idx) != 0,
+                SqlColumnType.Boolean => reader.GetByteAsBool(idx),
                 SqlColumnType.Int8 => reader.GetByte(idx),
                 SqlColumnType.Int16 => reader.GetShort(idx),
                 SqlColumnType.Int32 => reader.GetInt(idx),
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/BinaryTupleMethods.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/BinaryTupleMethods.cs
index 1d8ee61126..9eb40ba9fd 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/BinaryTupleMethods.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/BinaryTupleMethods.cs
@@ -36,6 +36,11 @@ namespace Apache.Ignite.Internal.Table.Serialization
         public static readonly MethodInfo WriteNoValue =
             typeof(BinaryTupleBuilderExtensions).GetMethod(nameof(BinaryTupleBuilderExtensions.AppendNoValue))!;
 
+        /// <summary>
+        /// <see cref="BinaryTupleReader.IsNull"/>.
+        /// </summary>
+        public static readonly MethodInfo IsNull = typeof(BinaryTupleReader).GetMethod(nameof(BinaryTupleReader.IsNull))!;
+
         private static readonly MethodInfo AppendByte = typeof(BinaryTupleBuilder).GetMethod(nameof(BinaryTupleBuilder.AppendByte))!;
         private static readonly MethodInfo AppendShort = typeof(BinaryTupleBuilder).GetMethod(nameof(BinaryTupleBuilder.AppendShort))!;
         private static readonly MethodInfo AppendInt = typeof(BinaryTupleBuilder).GetMethod(nameof(BinaryTupleBuilder.AppendInt))!;
@@ -55,6 +60,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
             typeof(BinaryTupleBuilder).GetMethod(nameof(BinaryTupleBuilder.AppendBytes), new[] { typeof(byte[]) })!;
 
         private static readonly MethodInfo GetByte = typeof(BinaryTupleReader).GetMethod(nameof(BinaryTupleReader.GetByte))!;
+        private static readonly MethodInfo GetByteAsBool = typeof(BinaryTupleReader).GetMethod(nameof(BinaryTupleReader.GetByteAsBool))!;
         private static readonly MethodInfo GetShort = typeof(BinaryTupleReader).GetMethod(nameof(BinaryTupleReader.GetShort))!;
         private static readonly MethodInfo GetInt = typeof(BinaryTupleReader).GetMethod(nameof(BinaryTupleReader.GetInt))!;
         private static readonly MethodInfo GetLong = typeof(BinaryTupleReader).GetMethod(nameof(BinaryTupleReader.GetLong))!;
@@ -95,6 +101,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
         {
             { typeof(string), GetString },
             { typeof(sbyte), GetByte },
+            { typeof(bool), GetByteAsBool },
             { typeof(short), GetShort },
             { typeof(int), GetInt },
             { typeof(long), GetLong },
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/ILGeneratorExtensions.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/ILGeneratorExtensions.cs
new file mode 100644
index 0000000000..a08843f463
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/ILGeneratorExtensions.cs
@@ -0,0 +1,78 @@
+/*
+ * 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.Ignite.Internal.Table.Serialization;
+
+using System;
+using System.Reflection;
+using System.Reflection.Emit;
+
+/// <summary>
+/// Extensions for <see cref="ILGenerator"/>.
+/// </summary>
+internal static class ILGeneratorExtensions
+{
+    /// <summary>
+    /// Declares a local and initializes with a default value (all fields zero).
+    /// </summary>
+    /// <param name="il">IL generator.</param>
+    /// <param name="type">Type.</param>
+    /// <returns>Resulting builder.</returns>
+    public static LocalBuilder DeclareAndInitLocal(this ILGenerator il, Type type)
+    {
+        var local = il.DeclareLocal(type);
+
+        if (type.IsValueType)
+        {
+            il.Emit(OpCodes.Ldloca_S, local);
+            il.Emit(OpCodes.Initobj, type);
+        }
+        else
+        {
+            il.Emit(OpCodes.Ldtoken, type);
+            il.Emit(OpCodes.Call, ReflectionUtils.GetTypeFromHandleMethod);
+            il.Emit(OpCodes.Call, ReflectionUtils.GetUninitializedObjectMethod);
+            il.Emit(OpCodes.Stloc, local);
+        }
+
+        return local;
+    }
+
+    /// <summary>
+    /// Emits corresponding Conv op code.
+    /// </summary>
+    /// <param name="il">IL generator.</param>
+    /// <param name="from">Source type.</param>
+    /// <param name="to">Target type.</param>
+    public static void EmitConv(this ILGenerator il, Type from, Type to)
+    {
+        if (from == to)
+        {
+            return;
+        }
+
+        var methodName = "To" + to.Name;
+        var method = typeof(Convert).GetMethod(methodName, BindingFlags.Static | BindingFlags.Public, new[] { from });
+
+        if (method == null)
+        {
+            throw new NotSupportedException($"Conversion from {from} to {to} is not supported.");
+        }
+
+        il.Emit(OpCodes.Call, method);
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/ObjectSerializerHandler.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/ObjectSerializerHandler.cs
index a0cfe9450d..8402a28bcf 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/ObjectSerializerHandler.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Serialization/ObjectSerializerHandler.cs
@@ -121,7 +121,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
 
                 if (col.Type == ClientDataType.Decimal)
                 {
-                    EmitLdcI4(il, col.Scale);
+                    il.Emit(OpCodes.Ldc_I4, col.Scale);
                 }
 
                 il.Emit(OpCodes.Call, directWriteMethod);
@@ -160,7 +160,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
 
                     if (col.Type == ClientDataType.Decimal)
                     {
-                        EmitLdcI4(il, col.Scale);
+                        il.Emit(OpCodes.Ldc_I4, col.Scale);
                     }
 
                     var writeMethod = BinaryTupleMethods.GetWriteMethod(fieldInfo.FieldType);
@@ -236,7 +236,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
 
                     if (col.Type == ClientDataType.Decimal)
                     {
-                        EmitLdcI4(il, col.Scale);
+                        il.Emit(OpCodes.Ldc_I4, col.Scale);
                     }
 
                     var writeMethod = BinaryTupleMethods.GetWriteMethod(fieldInfo.FieldType);
@@ -279,7 +279,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
 
                 if (schema.Columns[0] is { Type: ClientDataType.Decimal } col)
                 {
-                    EmitLdcI4(il, col.Scale);
+                    il.Emit(OpCodes.Ldc_I4, col.Scale);
                 }
 
                 il.Emit(OpCodes.Call, readMethod);
@@ -288,7 +288,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
                 return (ReadDelegate<T>)method.CreateDelegate(typeof(ReadDelegate<T>));
             }
 
-            var local = DeclareAndInitLocal(il, type);
+            var local = il.DeclareAndInitLocal(type);
 
             var columns = schema.Columns;
             var count = keyOnly ? schema.KeyColumnCount : columns.Count;
@@ -317,9 +317,9 @@ namespace Apache.Ignite.Internal.Table.Serialization
             var keyMethod = BinaryTupleMethods.GetReadMethodOrNull(keyType);
             var valMethod = BinaryTupleMethods.GetReadMethodOrNull(valType);
 
-            var kvLocal = DeclareAndInitLocal(il, type);
-            var keyLocal = keyMethod == null ? DeclareAndInitLocal(il, keyType) : null;
-            var valLocal = valMethod == null ? DeclareAndInitLocal(il, valType) : null;
+            var kvLocal = il.DeclareAndInitLocal(type);
+            var keyLocal = keyMethod == null ? il.DeclareAndInitLocal(keyType) : null;
+            var valLocal = valMethod == null ? il.DeclareAndInitLocal(valType) : null;
 
             var columns = schema.Columns;
             var count = keyOnly ? schema.KeyColumnCount : columns.Count;
@@ -396,7 +396,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
             }
 
             var il = method.GetILGenerator();
-            var local = DeclareAndInitLocal(il, type); // T res
+            var local = il.DeclareAndInitLocal(type); // T res
 
             var columns = schema.Columns;
 
@@ -433,7 +433,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
             var (_, valType, _, valField) = GetKeyValTypes();
 
             var il = method.GetILGenerator();
-            var kvLocal = DeclareAndInitLocal(il, type);
+            var kvLocal = il.DeclareAndInitLocal(type);
 
             var valReadMethod = BinaryTupleMethods.GetReadMethodOrNull(valType);
 
@@ -450,7 +450,7 @@ namespace Apache.Ignite.Internal.Table.Serialization
             }
             else
             {
-                var valLocal = DeclareAndInitLocal(il, valType);
+                var valLocal = il.DeclareAndInitLocal(valType);
                 var columns = schema.Columns;
 
                 for (var i = schema.KeyColumnCount; i < columns.Count; i++)
@@ -486,63 +486,18 @@ namespace Apache.Ignite.Internal.Table.Serialization
 
             il.Emit(local.LocalType.IsValueType ? OpCodes.Ldloca_S : OpCodes.Ldloc, local); // res
             il.Emit(OpCodes.Ldarg_0); // reader
-            EmitLdcI4(il, elemIdx); // index
+            il.Emit(OpCodes.Ldc_I4, elemIdx); // index
 
             if (col.Type == ClientDataType.Decimal)
             {
-                EmitLdcI4(il, col.Scale);
+                il.Emit(OpCodes.Ldc_I4, col.Scale);
             }
 
+            // TODO IGNITE-18329 Handle nullable types.
             il.Emit(OpCodes.Call, readMethod);
             il.Emit(OpCodes.Stfld, fieldInfo); // res.field = value
         }
 
-        private static void EmitLdcI4(ILGenerator il, int val)
-        {
-            switch (val)
-            {
-                case 0:
-                    il.Emit(OpCodes.Ldc_I4_0);
-                    break;
-
-                case 1:
-                    il.Emit(OpCodes.Ldc_I4_1);
-                    break;
-
-                case 2:
-                    il.Emit(OpCodes.Ldc_I4_2);
-                    break;
-
-                case 3:
-                    il.Emit(OpCodes.Ldc_I4_3);
-                    break;
-
-                case 4:
-                    il.Emit(OpCodes.Ldc_I4_4);
-                    break;
-
-                case 5:
-                    il.Emit(OpCodes.Ldc_I4_5);
-                    break;
-
-                case 6:
-                    il.Emit(OpCodes.Ldc_I4_6);
-                    break;
-
-                case 7:
-                    il.Emit(OpCodes.Ldc_I4_7);
-                    break;
-
-                case 8:
-                    il.Emit(OpCodes.Ldc_I4_8);
-                    break;
-
-                default:
-                    il.Emit(OpCodes.Ldc_I4, val);
-                    break;
-            }
-        }
-
         private static void ValidateFieldType(FieldInfo fieldInfo, Column column)
         {
             var columnType = column.Type.ToType();
@@ -583,26 +538,6 @@ namespace Apache.Ignite.Internal.Table.Serialization
                 $"Can't map '{type}' to columns '{columnStr}'. Matching fields not found.");
         }
 
-        private static LocalBuilder DeclareAndInitLocal(ILGenerator il, Type type)
-        {
-            var local = il.DeclareLocal(type);
-
-            if (type.IsValueType)
-            {
-                il.Emit(OpCodes.Ldloca_S, local);
-                il.Emit(OpCodes.Initobj, type);
-            }
-            else
-            {
-                il.Emit(OpCodes.Ldtoken, type);
-                il.Emit(OpCodes.Call, ReflectionUtils.GetTypeFromHandleMethod);
-                il.Emit(OpCodes.Call, ReflectionUtils.GetUninitializedObjectMethod);
-                il.Emit(OpCodes.Stloc, local);
-            }
-
-            return local;
-        }
-
         private static (Type KeyType, Type ValType, FieldInfo KeyField, FieldInfo ValField) GetKeyValTypes()
         {
             var type = typeof(T);