You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tajo.apache.org by hyunsik <gi...@git.apache.org> on 2014/07/14 07:36:22 UTC

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

GitHub user hyunsik opened a pull request:

    https://github.com/apache/tajo/pull/71

    TAJO-939: Refactoring the column resolver in LogicalPlan

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/hyunsik/tajo TAJO-939

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/tajo/pull/71.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #71
    
----
commit 8ff4d31297267547cf9111d6ac9fb9ed29e6e859
Author: Hyunsik Choi <hy...@apache.org>
Date:   2014-07-13T13:25:03Z

    Column Resolver should play different roles at each operation:
    
    First refactoring of resolver.

commit 048c953768da9943c5d87c3f2e680f57e60aa571
Author: Hyunsik Choi <hy...@apache.org>
Date:   2014-07-13T17:45:23Z

    Column Resolver should play different roles at each operation:
    
    Refactored many of createEval to use NameResolver.

commit cce2a566c1bca84d19e328d3b920c3b96af7bfe9
Author: Hyunsik Choi <hy...@apache.org>
Date:   2014-07-13T18:28:29Z

    Column Resolver should play different roles at each operation:
    
    Refactored CreateEval() except for join.

commit 2d1a1d81631056711d5ff91d92f0f934c9561525
Author: Hyunsik Choi <hy...@apache.org>
Date:   2014-07-14T02:15:29Z

    Column Resolver should play different roles at each operation:
    
    Completely extracted the column resolver from LogicalPlan.

commit ce12386d5e188679f48042ddc4262de59f9633be
Author: Hyunsik Choi <hy...@apache.org>
Date:   2014-07-14T04:53:42Z

    add comments.

commit 0cac367998ce0399d0e31e6b5669ba74e1bb91a8
Author: Hyunsik Choi <hy...@apache.org>
Date:   2014-07-14T05:09:37Z

    added more unit tests.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-48889103
  
    Hi Hyunsik, I have a question.
    Do you have any rules for resolving names?
    If so, please share it. It will help us reviewing your patch.
    Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14873359
  
    --- Diff: tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanPreprocessor.java ---
    @@ -0,0 +1,23 @@
    +/**
    + * 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.
    + */
    +
    +package org.apache.tajo.engine.planner;
    +
    +
    +public class TestLogicalPlanPreprocessor {
    --- End diff --
    
    It seems that you intend to add some tests, but they are missed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-48890108
  
    Thank you for the quick review. 
    
    Unfortunately, I couldn't find the rule, and I expect that it is not specified in SQL standards because  commercial DBMSs have different behaviors. 
    
    So, I analyzed the behavior of PostgreSQL, and then I made it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14873320
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -1192,8 +1204,12 @@ public ScanNode visitRelation(PlanContext context, Stack<Expr> stack, Relation e
         Set<String> newlyEvaluatedExprsReferences = new LinkedHashSet<String>();
         for (Iterator<NamedExpr> iterator = block.namedExprsMgr.getIteratorForUnevaluatedExprs(); iterator.hasNext();) {
           NamedExpr rawTarget = iterator.next();
    +//      if (rawTarget.getExpr().getType() == OpType.Column) {
    --- End diff --
    
    Please remove commented out codes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/tajo/pull/71


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14946827
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -712,7 +718,8 @@ public LimitNode visitLimit(PlanContext context, Stack<Expr> stack, Limit limit)
         EvalNode firstFetNum;
         LogicalNode child;
         if (limit.getFetchFirstNum().getType() == OpType.Literal) {
    -      firstFetNum = exprAnnotator.createEvalNode(context.plan, block, limit.getFetchFirstNum());
    +      firstFetNum = exprAnnotator.createEvalNode(context.plan, block, limit.getFetchFirstNum(),
    +          NameResolvingMode.RELS_ONLY);
    --- End diff --
    
    Would you explain why RELS_ONLY is used for limit, please?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r15056936
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/nameresolver/NameResolvingMode.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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.
    + */
    +
    +package org.apache.tajo.engine.planner.nameresolver;
    +
    +/**
    + *
    + * <h2>Motivation</h2>
    + *
    + * Please take a look at the following example query:
    + *
    + *  <pre>
    + *   select (l_orderkey + l_orderkey) l_orderkey from lineitem where l_orderkey > 2 order by l_orderkey;
    + * </pre>
    + *
    + * Although <code>l_orderkey</code> seems to be ambiguous, the above usages are available in commercial DBMSs.
    + * In order to eliminate the ambiguity, Tajo follows the behaviors of PostgreSQL.
    + *
    + * <h2>Resolving Modes</h2>
    + *
    + * From the behaviors of PostgreSQL, we found that there are three kinds of name resolving modes.
    + * Each definition is as follows:
    + *
    + * <ul>
    + *   <li><b>RELS_ONLY</b> finds a column from the relations in the current block.
    + *  <li><b>RELS_AND_SUBEXPRS</b> finds a column from the all relations in the current block and
    + *  from aliased temporal fields; a temporal field means an explicitly aliased expression. If there are duplicated
    + *  columns in the relation and temporal fields, this level firstly chooses the field in a relation.</li>
    + *  <li><b>SUBEXPRS_AND_RELS</b> is very similar to <code>RELS_AND_SUBEXPRS</code>. The main difference is that it
    + *  firstly chooses an aliased temporal field instead of the fields in a relation.</li>
    + * </ul>
    + *
    + * <h2>The relationship between resolving modes and operators</h3>
    + *
    + * <ul>
    + *   <li>fields in select list are resolved in the REL_ONLY mode.</li>
    + *   <li>fields in WHERE clause are resolved in the RELS_AND_SUBEXPRS mode.</li>
    + *   <li>fields in GROUP BY, HAVING, ORDER BY, and LIMIT are resolved in the SUBEXPRS_AND_RELS mode.</li>
    --- End diff --
    
    As you commented, fields in LIMIT are resolved in the REL_ONLY mode. Please fix it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r15042369
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -712,7 +718,8 @@ public LimitNode visitLimit(PlanContext context, Stack<Expr> stack, Limit limit)
         EvalNode firstFetNum;
         LogicalNode child;
         if (limit.getFetchFirstNum().getType() == OpType.Literal) {
    -      firstFetNum = exprAnnotator.createEvalNode(context.plan, block, limit.getFetchFirstNum());
    +      firstFetNum = exprAnnotator.createEvalNode(context.plan, block, limit.getFetchFirstNum(),
    +          NameResolvingMode.RELS_ONLY);
    --- End diff --
    
    LIMIT has only constant. It does not have column reference. Its reason is the same that I commented for non-from statement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r15042347
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -383,7 +386,7 @@ private EvalExprNode buildPlanForNoneFromStatement(PlanContext context, Stack<Ex
     
         for (int i = 0; i < targets.length; i++) {
           NamedExpr namedExpr = projection.getNamedExprs()[i];
    -      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
    +      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr(), NameResolvingMode.RELS_ONLY);
    --- End diff --
    
    As you know, non-from statement does not need to resolve any column because non-from statement means no table and no column reference. 
    
    The main difference among the resolving modes is the priority and scope. It is enough to use the mode which has the correct priority and the scope. If possible, it would be nice if we use the mode the minimum scope. RELS_ONLY is the minimum scope one.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-49688721
  
    @jihoonson Join requires more complex conditions. So, I need to clean up the name resolving logic for join condition, and I need more time for this job. If you are Ok, I'll do it in another Jira.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14946624
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -383,7 +386,7 @@ private EvalExprNode buildPlanForNoneFromStatement(PlanContext context, Stack<Ex
     
         for (int i = 0; i < targets.length; i++) {
           NamedExpr namedExpr = projection.getNamedExprs()[i];
    -      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
    +      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr(), NameResolvingMode.RELS_ONLY);
    --- End diff --
    
    Can RESL_ONLY level be applied for non-from statement?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-49688899
  
    Rebased.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-48907516
  
    Please see the following comments. I've just added the explanation about the motivation, all modes, and an example.
    https://github.com/apache/tajo/pull/71/files#diff-4f557feed9e12061e60da8dc979e7a5bR23
    
    I hope that it's helpful for your understanding.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-49550930
  
    @jihoonson I really appreciate your detailed review. I rebased the branch against the latest revision, and I reflected your comment. Please help to review it again. 
    
    Thank you!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-49709961
  
    I've made the jira to clean up the name resolving logic for join condition.
    https://issues.apache.org/jira/browse/TAJO-971


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-49690843
  
    Thanks for your reply!
    Currently, the latest patch looks good, and +1!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14873857
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -1192,8 +1204,12 @@ public ScanNode visitRelation(PlanContext context, Stack<Expr> stack, Relation e
         Set<String> newlyEvaluatedExprsReferences = new LinkedHashSet<String>();
         for (Iterator<NamedExpr> iterator = block.namedExprsMgr.getIteratorForUnevaluatedExprs(); iterator.hasNext();) {
           NamedExpr rawTarget = iterator.next();
    +//      if (rawTarget.getExpr().getType() == OpType.Column) {
    --- End diff --
    
    I'll remove it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-48890833
  
    Thanks for your reply. 
    If you can, would you mind sharing your rule?
    Actually, while reviewing, it is hard for me to check that resolve level is properly set. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r15056981
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -712,7 +718,8 @@ public LimitNode visitLimit(PlanContext context, Stack<Expr> stack, Limit limit)
         EvalNode firstFetNum;
         LogicalNode child;
         if (limit.getFetchFirstNum().getType() == OpType.Literal) {
    -      firstFetNum = exprAnnotator.createEvalNode(context.plan, block, limit.getFetchFirstNum());
    +      firstFetNum = exprAnnotator.createEvalNode(context.plan, block, limit.getFetchFirstNum(),
    +          NameResolvingMode.RELS_ONLY);
    --- End diff --
    
    Thanks Hyunsik!
    I misunderstood because it differs from the explanation of NameResolvingMode.
    I left another comment on it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-48981177
  
    Thanks!
    I'll review today.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14873254
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java ---
    @@ -197,6 +199,17 @@ public LogicalNode visitProjection(PreprocessContext ctx, Stack<Expr> stack, Pro
         }
     
         NamedExpr[] projectTargetExprs = expr.getNamedExprs();
    +    NameRefInSelectListNormalizer normalizer = new NameRefInSelectListNormalizer();
    +
    +    for (int i = 0; i < expr.getNamedExprs().length; i++) {
    +      NamedExpr namedExpr = expr.getNamedExprs()[i];
    --- End diff --
    
    It would be better use the above 'projectTargetExprs' variable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-49691285
  
    Thank you very much. I'll create the jira.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14873865
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java ---
    @@ -197,6 +199,17 @@ public LogicalNode visitProjection(PreprocessContext ctx, Stack<Expr> stack, Pro
         }
     
         NamedExpr[] projectTargetExprs = expr.getNamedExprs();
    +    NameRefInSelectListNormalizer normalizer = new NameRefInSelectListNormalizer();
    +
    +    for (int i = 0; i < expr.getNamedExprs().length; i++) {
    +      NamedExpr namedExpr = expr.getNamedExprs()[i];
    --- End diff --
    
    Thank you for your suggestion. I'll do so.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r15056956
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -383,7 +386,7 @@ private EvalExprNode buildPlanForNoneFromStatement(PlanContext context, Stack<Ex
     
         for (int i = 0; i < targets.length; i++) {
           NamedExpr namedExpr = projection.getNamedExprs()[i];
    -      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
    +      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr(), NameResolvingMode.RELS_ONLY);
    --- End diff --
    
    Thanks Hyunsik!
    I misunderstood because it differs from the explanation of NameResolvingMode.
    I left another comment on it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14873851
  
    --- Diff: tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanPreprocessor.java ---
    @@ -0,0 +1,23 @@
    +/**
    + * 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.
    + */
    +
    +package org.apache.tajo.engine.planner;
    +
    +
    +public class TestLogicalPlanPreprocessor {
    --- End diff --
    
    Oops, I'll remove it soon.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r15207047
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -260,7 +262,8 @@ private void setRawTargets(PlanContext context, Target[] targets, String[] refer
         Target [] rawTargets = new Target[projection.getNamedExprs().length];
         for (int i = 0; i < projection.getNamedExprs().length; i++) {
           NamedExpr namedExpr = projection.getNamedExprs()[i];
    -      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
    +      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr(),
    --- End diff --
    
    name references in select list can be only actual column names. They are addressed in the manner at https://github.com/apache/tajo/pull/71/files#diff-c11c2244fea624cb7b08f6906d554cd4R470.
    
    In contrast, setRawTargets() in visitProjection should deal with actual fields as well as temporal fields generated by common subexpression elimination. So, it should use RELS_AND_SUBEXPRS.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/71#discussion_r14945799
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---
    @@ -260,7 +262,8 @@ private void setRawTargets(PlanContext context, Target[] targets, String[] refer
         Target [] rawTargets = new Target[projection.getNamedExprs().length];
         for (int i = 0; i < projection.getNamedExprs().length; i++) {
           NamedExpr namedExpr = projection.getNamedExprs()[i];
    -      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr());
    +      EvalNode evalNode = exprAnnotator.createEvalNode(plan, block, namedExpr.getExpr(),
    --- End diff --
    
    In setRawTargets(), raw expressions are found instead of references. So, it seems that resolving level should be set as RELS_ONLY. If I misunderstand, please let me know. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-49690877
  
    And please create a Jira issue for name resolution of join condition.
    Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-939: Refactoring the column resolver in Lo...

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the pull request:

    https://github.com/apache/tajo/pull/71#issuecomment-49059484
  
    Hi Hyunsik. I left some questions.
    Also, I have one more question.
    Would you mind explaining why legacy level is used for join? Is RELS_AND_SUBEXPRS sufficient such as where clause?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---