You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/04/06 13:31:49 UTC

[42/50] [abbrv] ignite git commit: IGNITE-4817 .NET: Fixed LINQ Contains when subquery comes from a variable

IGNITE-4817 .NET: Fixed LINQ Contains when subquery comes from a variable

This closes #1745


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

Branch: refs/heads/ignite-2893
Commit: 7c80c477d38c59cd64045c5eb9f048d367a7978d
Parents: 21d3475
Author: Sergey Stronchinskiy <gu...@gmail.com>
Authored: Thu Apr 6 10:31:19 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Apr 6 10:31:19 2017 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheLinqTest.cs                | 36 ++++++++--
 .../Impl/CacheQueryExpressionVisitor.cs         | 14 +++-
 .../Apache.Ignite.Linq/Impl/CacheQueryParser.cs | 25 ++++++-
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    | 72 ++++++++++++++++----
 4 files changed, 125 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7c80c477/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 856fb49..dfd644d 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
@@ -276,12 +276,21 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             CheckFunc(x => x.Trim(), strings);
             CheckFunc(x => x.Trim('P'), strings);
+            var toTrim = new[] {'P'};
+            CheckFunc(x => x.Trim(toTrim), strings);
+            CheckFunc(x => x.Trim(new List<char> {'P'}.ToArray()), strings);
             CheckFunc(x => x.Trim('3'), strings);
             CheckFunc(x => x.TrimStart('P'), strings);
+            CheckFunc(x => x.TrimStart(toTrim), strings);
             CheckFunc(x => x.TrimStart('3'), strings);
             Assert.Throws<NotSupportedException>(() => CheckFunc(x => x.TrimStart('P', 'e'), strings));
             CheckFunc(x => x.TrimEnd('P'), strings);
+            CheckFunc(x => x.TrimEnd(toTrim), strings);
             CheckFunc(x => x.TrimEnd('3'), strings);
+            var toTrimFails = new[] {'P', 'c'};
+            Assert.Throws<NotSupportedException>(() => CheckFunc(x => x.Trim(toTrimFails), strings));
+            Assert.Throws<NotSupportedException>(() => CheckFunc(x => x.TrimStart(toTrimFails), strings));
+            Assert.Throws<NotSupportedException>(() => CheckFunc(x => x.TrimEnd(toTrimFails), strings));
 
             CheckFunc(x => Regex.Replace(x, @"son.\d", "kele!"), strings);
             CheckFunc(x => x.Replace("son", ""), strings);
@@ -726,7 +735,8 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
                 .ThenBy(x => x.Value.Age)
                 .ToArray();
 
-            Assert.AreEqual(Enumerable.Range(0, PersonCount).Reverse().ToArray(), persons.Select(x => x.Key).ToArray());
+            Assert.AreEqual(Enumerable.Range(0, PersonCount).Reverse().ToArray(), 
+                persons.Select(x => x.Key).ToArray());
 
             var personsByOrg = GetPersonCache().AsCacheQueryable()
                 .Join(GetOrgCache().AsCacheQueryable(), p => p.Value.OrganizationId, o => o.Value.Id,
@@ -816,6 +826,22 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             var ex = Assert.Throws<NotSupportedException>(() =>
                 CompiledQuery.Compile((int[] k) => cache.Where(x => k.Contains(x.Key))));
             Assert.AreEqual("'Contains' clause coming from compiled query parameter is not supported.", ex.Message);
+
+            // check subquery from another cache put in separate variable
+            var orgIds = orgCache
+                .Where(o => o.Value.Name == "Org_1")
+                .Select(o => o.Key);
+
+            var subQueryFromVar = cache
+                .Where(x => orgIds.Contains(x.Value.OrganizationId))
+                .ToArray();
+
+            var subQueryInline = cache
+                .Where(x => orgCache.Where(o => o.Value.Name == "Org_1")
+                    .Select(o => o.Key).Contains(x.Value.OrganizationId))
+                .ToArray();
+
+            Assert.AreEqual(subQueryInline.Length, subQueryFromVar.Length);
         }
 
         /// <summary>
@@ -1047,7 +1073,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             var qry6 = CompiledQuery.Compile((int minAge) => persons
                 .Select(x => x.Value)
                 .Where(x => x.Age >= minAge)
-                .Select(x => new {x.Name, x.Age})
+                .Select(x => new { x.Name, x.Age })
                 .OrderBy(x => x.Name));
 
             var res = qry6(PersonCount - 3).GetAll();
@@ -1116,7 +1142,8 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             // 8 arg
             var qry8 = CompiledQuery.Compile((int a, int b, int c, int d, int e, int f, int g, int h) =>
-                cache.Select(x => x.Key).Where(k => k > a && k > b && k > c && k < d && k < e && k < f && k < g && k < h));
+                cache.Select(x => x.Key)
+                    .Where(k => k > a && k > b && k > c && k < d && k < e && k < f && k < g && k < h));
             Assert.AreEqual(new[] {3, 4}, qry8(0, 1, 2, 5, 6, 7, 8, 9).ToArray());
         }
 
@@ -1432,7 +1459,8 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         /// <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)
+        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

http://git-wip-us.apache.org/repos/asf/ignite/blob/7c80c477/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 94e59fa..99712e3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
@@ -93,26 +93,34 @@ namespace Apache.Ignite.Linq.Impl
         [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0")]
         protected override Expression VisitUnary(UnaryExpression expression)
         {
-            ResultBuilder.Append("(");
+            var closeBracket = false;
 
             switch (expression.NodeType)
             {
                 case ExpressionType.Negate:
+                    ResultBuilder.Append("(");
                     ResultBuilder.Append("-");
+                    closeBracket = true;
                     break;
+
                 case ExpressionType.Not:
+                    ResultBuilder.Append("(");
                     ResultBuilder.Append("not ");
+                    closeBracket = true;
                     break;
+
                 case ExpressionType.Convert:
                     // Ignore, let the db do the conversion
                     break;
+
                 default:
                     return base.VisitUnary(expression);
             }
 
             Visit(expression.Operand);
 
-            ResultBuilder.Append(")");
+            if(closeBracket)
+                ResultBuilder.Append(")");
 
             return expression;
         }
@@ -418,7 +426,7 @@ namespace Apache.Ignite.Linq.Impl
         /// <summary>
         /// Appends the parameter.
         /// </summary>
-        private void AppendParameter(object value)
+        public void AppendParameter(object value)
         {
             ResultBuilder.Append("?");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7c80c477/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryParser.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryParser.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryParser.cs
index cee90f4..794ef2e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryParser.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryParser.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Linq.Impl
 {
     using System.Threading;
     using Remotion.Linq.Parsing.ExpressionVisitors.Transformation;
+    using Remotion.Linq.Parsing.ExpressionVisitors.TreeEvaluation;
     using Remotion.Linq.Parsing.Structure;
     using Remotion.Linq.Parsing.Structure.ExpressionTreeProcessors;
 
@@ -46,11 +47,33 @@ namespace Apache.Ignite.Linq.Impl
         {
             var transformerRegistry = ExpressionTransformerRegistry.CreateDefault();
 
-            var proc = new TransformingExpressionTreeProcessor(transformerRegistry);
+            var proc = CreateCompoundProcessor(transformerRegistry);
 
             var parser = new ExpressionTreeParser(ExpressionTreeParser.CreateDefaultNodeTypeProvider(), proc);
 
             return new QueryParser(parser);
         }
+
+        /// <summary>
+        /// Creates CompoundExpressionTreeProcessor.
+        /// </summary>
+        private static CompoundExpressionTreeProcessor CreateCompoundProcessor(
+            IExpressionTranformationProvider tranformationProvider)
+        {
+            return new CompoundExpressionTreeProcessor(
+                new IExpressionTreeProcessor[]
+                {
+                    new PartialEvaluatingExpressionTreeProcessor(new NullEvaluatableExpressionFilter()),
+                    new TransformingExpressionTreeProcessor(tranformationProvider)
+                });
+        }
+
+        /// <summary>
+        /// Empty implementation of IEvaluatableExpressionFilter.
+        /// </summary>
+        private sealed class NullEvaluatableExpressionFilter : EvaluatableExpressionFilterBase
+        {
+            // No-op.
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7c80c477/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
index e83c448..578c5da 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
@@ -62,9 +62,9 @@ namespace Apache.Ignite.Linq.Impl
             GetStringMethod("Substring", new[] {typeof (int)}, GetFunc("substring", 0, 1)),
             GetStringMethod("Substring", new[] {typeof (int), typeof (int)}, GetFunc("substring", 0, 1)),
             GetStringMethod("Trim", "trim"),
-            GetStringMethod("Trim", "trim", typeof(char[])),
-            GetStringMethod("TrimStart", "ltrim", typeof(char[])),
-            GetStringMethod("TrimEnd", "rtrim", typeof(char[])),
+            GetParameterizedTrimMethod("Trim", "trim"),
+            GetParameterizedTrimMethod("TrimStart", "ltrim"),
+            GetParameterizedTrimMethod("TrimEnd", "rtrim"),
             GetStringMethod("Replace", "replace", typeof(string), typeof(string)),
 
             GetMethod(typeof (Regex), "Replace", new[] {typeof (string), typeof (string), typeof (string)}, 
@@ -176,25 +176,59 @@ namespace Apache.Ignite.Linq.Impl
                 if (isInstanceMethod || (i > 0))
                     visitor.ResultBuilder.Append(", ");
 
-                if (arg.NodeType == ExpressionType.NewArrayInit)
+                visitor.Visit(arg);
+
+                AppendAdjustment(visitor, adjust, i + 1);
+            }
+
+            visitor.ResultBuilder.Append(suffix).Append(")");
+
+            AppendAdjustment(visitor, adjust, 0);
+        }
+
+        /// <summary>
+        /// Visits the instance function for Trim specific handling.
+        /// </summary>
+        private static void VisitParameterizedTrimFunc(MethodCallExpression expression,
+            CacheQueryExpressionVisitor visitor, string func)
+        {
+            visitor.ResultBuilder.Append(func).Append("(");
+
+            visitor.Visit(expression.Object);
+
+            var arg = expression.Arguments[0];
+
+            if (arg != null)
+            {
+                visitor.ResultBuilder.Append(", ");
+
+                if (arg.NodeType == ExpressionType.Constant)
                 {
-                    // Only trim methods use params[], only one param is supported
-                    var args = ((NewArrayExpression) arg).Expressions;
+                    var constant = (ConstantExpression) arg;
+                    var args = constant.Value as IEnumerable<char>;
+
+                    if (args == null)
+                    {
+                        throw new NotSupportedException("String.Trim function only supports IEnumerable<char>");
+                    }
+
+                    var enumeratedArgs = args.ToArray();
 
-                    if (args.Count != 1)
-                        throw new NotSupportedException("Method call only supports a single parameter: "+ expression);
+                    if (enumeratedArgs.Length != 1)
+                    {
+                        throw new NotSupportedException("String.Trim function only supports a single argument: " +
+                                                        expression);
+                    }
 
-                    visitor.Visit(args[0]);
+                    visitor.AppendParameter(enumeratedArgs[0]);
                 }
                 else
+                {
                     visitor.Visit(arg);
-
-                AppendAdjustment(visitor, adjust, i + 1);
+                }
             }
 
-            visitor.ResultBuilder.Append(suffix).Append(")");
-
-            AppendAdjustment(visitor, adjust, 0);
+            visitor.ResultBuilder.Append(")");
         }
 
         /// <summary>
@@ -261,6 +295,16 @@ namespace Apache.Ignite.Linq.Impl
         }
 
         /// <summary>
+        /// Gets string parameterized Trim(TrimStart, TrimEnd) method.
+        /// </summary>
+        private static KeyValuePair<MethodInfo, VisitMethodDelegate> GetParameterizedTrimMethod(string name,
+            string sqlName)
+        {
+            return GetMethod(typeof(string), name, new[] {typeof(char[])}, 
+                (e, v) => VisitParameterizedTrimFunc(e, v, sqlName));
+        }
+
+        /// <summary>
         /// Gets the math method.
         /// </summary>
         private static KeyValuePair<MethodInfo, VisitMethodDelegate> GetMathMethod(string name, string sqlName,