You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2017/02/16 17:36:28 UTC

[27/50] [abbrv] ignite git commit: IGNITE-4425 .NET: Support "ICollection.Contains" in LINQ

IGNITE-4425 .NET: Support "ICollection.Contains" in LINQ

This closes #1502


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d949b739
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d949b739
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d949b739

Branch: refs/heads/ignite-comm-balance-master
Commit: d949b739d1fc47a13dcbf8fe107276bc603b6d92
Parents: 2f4bdbb
Author: Sergey Stronchinskiy <gu...@gmail.com>
Authored: Fri Feb 10 15:38:01 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Feb 10 15:38:52 2017 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheLinqTest.cs                |  96 ++++++++++++++
 .../Impl/CacheQueryExpressionVisitor.cs         | 127 ++++++++++++++++++-
 .../Impl/CacheQueryModelVisitor.cs              |   3 +
 3 files changed, 224 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d949b739/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index 798e7e8..931fdd4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -28,6 +28,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 {
     using System;
     using System.Collections;
+    using System.Collections.Generic;
     using System.Linq;
     using System.Linq.Expressions;
     using System.Text.RegularExpressions;
@@ -106,6 +107,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             orgCache[1000] = new Organization {Id = 1000, Name = "Org_0"};
             orgCache[1001] = new Organization {Id = 1001, Name = "Org_1"};
+            orgCache[1002] = new Organization {Id = 1002, Name = null};
 
             var roleCache = GetRoleCache();
 
@@ -747,6 +749,74 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests IEnumerable.Contains.
+        /// </summary>
+        [Test]
+        public void TestContains()
+        {
+            var cache = GetPersonCache().AsCacheQueryable();
+            var orgCache = GetOrgCache().AsCacheQueryable();
+
+            var keys = new[] { 1, 2, 3 };
+            var emptyKeys = new int[0];
+
+            var bigNumberOfKeys = 10000;
+            var aLotOfKeys = Enumerable.Range(-bigNumberOfKeys + 10 - PersonCount, bigNumberOfKeys + PersonCount)
+                .ToArray();
+            var hashSetKeys = new HashSet<int>(keys);
+
+            CheckWhereFunc(cache, e => new[] { 1, 2, 3 }.Contains(e.Key));
+            CheckWhereFunc(cache, e => emptyKeys.Contains(e.Key));
+            CheckWhereFunc(cache, e => new int[0].Contains(e.Key));
+            CheckWhereFunc(cache, e => new int[0].Contains(e.Key));
+            CheckWhereFunc(cache, e => new List<int> { 1, 2, 3 }.Contains(e.Key));
+            CheckWhereFunc(cache, e => new List<int>(keys).Contains(e.Key));
+            CheckWhereFunc(cache, e => aLotOfKeys.Contains(e.Key));
+            CheckWhereFunc(cache, e => hashSetKeys.Contains(e.Key));
+            CheckWhereFunc(cache, e => !keys.Contains(e.Key));
+            CheckWhereFunc(orgCache, e => new[] { "Org_1", "NonExistentName", null }.Contains(e.Value.Name));
+            CheckWhereFunc(orgCache, e => !new[] { "Org_1", "NonExistentName", null }.Contains(e.Value.Name));
+            CheckWhereFunc(orgCache, e => new[] { "Org_1", null, null }.Contains(e.Value.Name));
+            CheckWhereFunc(orgCache, e => !new[] { "Org_1", null, null }.Contains(e.Value.Name));
+            CheckWhereFunc(orgCache, e => new string[] { null }.Contains(e.Value.Name));
+            CheckWhereFunc(orgCache, e => !new string[] { null }.Contains(e.Value.Name));
+            CheckWhereFunc(orgCache, e => !new string[] { null, null }.Contains(e.Value.Name));
+            CheckWhereFunc(orgCache, e => new string[] { null, null }.Contains(e.Value.Name));
+
+            //check passing a null object as collection
+            int[] nullKeys = null;
+            var nullKeysEntries = cache
+                .Where(e => nullKeys.Contains(e.Key))
+                .ToArray();
+            Assert.AreEqual(0, nullKeysEntries.Length, "Evaluating 'null.Contains' should return zero results");
+
+
+            Func<int[]> getKeysFunc = () => null;
+            var funcNullKeyEntries = cache
+                .Where(e => getKeysFunc().Contains(e.Key))
+                .ToArray();
+            Assert.AreEqual(0, funcNullKeyEntries.Length, "Evaluating 'null.Contains' should return zero results");
+
+
+            // Check subselect from other cache
+            var subSelectCount = cache
+                .Count(entry => orgCache
+                    .Where(orgEntry => orgEntry.Value.Name == "Org_1")
+                    .Select(orgEntry => orgEntry.Key)
+                    .Contains(entry.Value.OrganizationId));
+            var orgNumberOne = orgCache
+                .Where(orgEntry => orgEntry.Value.Name == "Org_1")
+                .Select(orgEntry => orgEntry.Key)
+                .First();
+            var subSelectCheckCount = cache.Count(entry => entry.Value.OrganizationId == orgNumberOne);
+            Assert.AreEqual(subSelectCheckCount, subSelectCount, "subselecting another CacheQueryable failed");
+
+            var ex = Assert.Throws<NotSupportedException>(() =>
+                CompiledQuery2.Compile((int[] k) => cache.Where(x => k.Contains(x.Key))));
+            Assert.AreEqual("'Contains' clause coming from compiled query parameter is not supported.", ex.Message);
+        }
+
+        /// <summary>
         /// Tests nulls.
         /// </summary>
         [Test]
@@ -1412,6 +1482,32 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             CollectionAssert.AreEqual(expected, actual, new NumericComparer());
         }
 
+        /// <summary>
+        /// Checks that function used in Where Clause maps to SQL function properly
+        /// </summary>
+        private static void CheckWhereFunc<TKey, TEntry>(IQueryable<ICacheEntry<TKey,TEntry>> query, Expression<Func<ICacheEntry<TKey, TEntry>,bool>> whereExpression)
+        {
+            // Calculate result locally, using real method invocation
+            var expected = query
+                .ToArray()
+                .AsQueryable()
+                .Where(whereExpression)
+                .Select(entry => entry.Key)
+                .OrderBy(x => x)
+                .ToArray();
+
+            // Perform SQL query
+            var actual = query
+                .Where(whereExpression)
+                .Select(entry => entry.Key)
+                .ToArray()
+                .OrderBy(x => x)
+                .ToArray();
+
+            // Compare results
+            CollectionAssert.AreEqual(expected, actual, new NumericComparer());
+        }
+
         public interface IPerson
         {
             int Age { get; set; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d949b739/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
index 1f9da1c..94e59fa 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
@@ -20,6 +20,7 @@ using System.Text;
 
 namespace Apache.Ignite.Linq.Impl
 {
+    using System.Collections;
     using System.Collections.Generic;
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
@@ -29,6 +30,7 @@ namespace Apache.Ignite.Linq.Impl
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Impl.Common;
+    using Remotion.Linq;
     using Remotion.Linq.Clauses;
     using Remotion.Linq.Clauses.Expressions;
     using Remotion.Linq.Clauses.ResultOperators;
@@ -470,15 +472,136 @@ namespace Apache.Ignite.Linq.Impl
         }
 
         /** <inheritdoc /> */
+
         [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         protected override Expression VisitSubQuery(SubQueryExpression expression)
         {
-            // This happens when New expression uses a subquery, in a GroupBy.
-            _modelVisitor.VisitSelectors(expression.QueryModel, false);
+            var subQueryModel = expression.QueryModel;
+
+            var contains = subQueryModel.ResultOperators.FirstOrDefault() as ContainsResultOperator;
+            
+            // Check if IEnumerable.Contains is used.
+            if (subQueryModel.ResultOperators.Count == 1 && contains != null)
+            {
+                VisitContains(subQueryModel, contains);
+            }
+            else
+            {
+                // This happens when New expression uses a subquery, in a GroupBy.
+                _modelVisitor.VisitSelectors(expression.QueryModel, false);
+            }
 
             return expression;
         }
 
+        /// <summary>
+        /// Visits IEnumerable.Contains
+        /// </summary>
+        private void VisitContains(QueryModel subQueryModel, ContainsResultOperator contains)
+        {
+            ResultBuilder.Append("(");
+
+            var fromExpression = subQueryModel.MainFromClause.FromExpression;
+
+            var queryable = ExpressionWalker.GetCacheQueryable(fromExpression, false);
+
+            if (queryable != null)
+            {
+                Visit(contains.Item);
+
+                ResultBuilder.Append(" IN (");
+                _modelVisitor.VisitQueryModel(subQueryModel);
+                ResultBuilder.Append(")");
+            }
+            else
+            {
+                var inValues = GetInValues(fromExpression).ToArray();
+
+                var hasNulls = inValues.Any(o => o == null);
+
+                if (hasNulls)
+                {
+                    ResultBuilder.Append("(");
+                }
+
+                Visit(contains.Item);
+
+                ResultBuilder.Append(" IN (");
+                AppendInParameters(inValues);
+                ResultBuilder.Append(")");
+
+                if (hasNulls)
+                {
+                    ResultBuilder.Append(") OR ");
+                    Visit(contains.Item);
+                    ResultBuilder.Append(" IS NULL");
+                }
+            }
+
+            ResultBuilder.Append(")");
+        }
+
+        /// <summary>
+        /// Gets values for IN expression.
+        /// </summary>
+        private static IEnumerable<object> GetInValues(Expression fromExpression)
+        {
+            IEnumerable result;
+            switch (fromExpression.NodeType)
+            {
+                case ExpressionType.MemberAccess:
+                    var memberExpression = (MemberExpression) fromExpression;
+                    result = ExpressionWalker.EvaluateExpression<IEnumerable>(memberExpression);
+                    break;
+                case ExpressionType.ListInit:
+                    var listInitExpression = (ListInitExpression) fromExpression;
+                    result = listInitExpression.Initializers
+                        .SelectMany(init => init.Arguments)
+                        .Select(ExpressionWalker.EvaluateExpression<object>);
+                    break;
+                case ExpressionType.NewArrayInit:
+                    var newArrayExpression = (NewArrayExpression) fromExpression;
+                    result = newArrayExpression.Expressions
+                        .Select(ExpressionWalker.EvaluateExpression<object>);
+                    break;
+                case ExpressionType.Parameter:
+                    // This should happen only when 'IEnumerable.Contains' is called on parameter of compiled query
+                    throw new NotSupportedException("'Contains' clause coming from compiled query parameter is not supported.");
+                default:
+                    result = Expression.Lambda(fromExpression).Compile().DynamicInvoke() as IEnumerable;
+                    break;
+            }
+
+            result = result ?? Enumerable.Empty<object>();
+
+            return result
+                .Cast<object>()
+                .ToArray();
+        }
+
+        /// <summary>
+        /// Appends not null parameters using ", " as delimeter.
+        /// </summary>
+        private void AppendInParameters(IEnumerable<object> values)
+        {
+            var first = true;
+
+            foreach (var val in values)
+            {
+                if (val == null)
+                    continue;
+
+                if (!first)
+                {
+                    ResultBuilder.Append(", ");
+                }
+
+                first = false;
+
+                AppendParameter(val);
+            }
+        }
+
         /** <inheritdoc /> */
         protected override Exception CreateUnhandledItemException<T>(T unhandledItem, string visitMethod)
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d949b739/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs
index ae94cfb..7cc9265 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs
@@ -184,6 +184,9 @@ namespace Apache.Ignite.Linq.Impl
                          || op is DefaultIfEmptyResultOperator || op is SkipResultOperator || op is TakeResultOperator)
                     // Will be processed later
                     break;
+                else if (op is ContainsResultOperator)
+                    // Should be processed already
+                    break;
                 else
                     throw new NotSupportedException("Operator is not supported: " + op);
             }