You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tajo.apache.org by jihoonson <gi...@git.apache.org> on 2014/11/09 15:47:51 UTC

[GitHub] tajo pull request: Tajo 838 3

GitHub user jihoonson opened a pull request:

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

    Tajo 838 3

    The previous work is extended to support indexes for multiple columns or even expressions. Even though we don't provide any methods to estimate the query selectivity, users can use the index scan by configuring the session variables. Finally, I have added more unit tests including performing join, aggregation, sort after index scans.
    
    As summarized below, some necessary features are still lacked in this work.
    However, IMHO, this work is ready to be merged into master branch as a prototype of query execution using index. Please leave any comments.
    Following features should be added.
    * Query optimization based on selectivity estimation
    * Support index for partitioned tables

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

    $ git pull https://github.com/jihoonson/tajo-2 TAJO-838_3

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

    https://github.com/apache/tajo/pull/231.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 #231
    
----
commit bbaf5b003ce29525ee6789fe0d7c57bd4b7390b6
Author: Jihoon Son <ji...@apache.org>
Date:   2014-10-26T07:43:34Z

    TAJO-838_3

commit ef4007404dcfca09a59f9356afad812855b636d0
Author: Jihoon Son <ji...@apache.org>
Date:   2014-10-26T07:56:07Z

    TAJO-838_3

commit fb55d3304324c61e045c82bacfb118a13757caab
Author: Jihoon Son <ji...@apache.org>
Date:   2014-10-27T09:02:00Z

    Merge branch 'index_support' of http://git-wip-us.apache.org/repos/asf/tajo into TAJO-838_3

commit aa57650fb1bb33ed4cb605301e991bad9ff863ef
Author: Jihoon Son <ji...@apache.org>
Date:   2014-10-28T02:21:10Z

    Merge branch 'index_support' of http://git-wip-us.apache.org/repos/asf/tajo into TAJO-838_3

commit 863ebeabead8904b739714e6b54233018c8d7e30
Author: Jihoon Son <ji...@apache.org>
Date:   2014-10-29T05:49:48Z

    Merge branch 'index_support' of http://git-wip-us.apache.org/repos/asf/tajo into TAJO-838_3

commit cbf0d84f670d509fd396ba636a23c9599e11da6d
Author: Jihoon Son <ji...@apache.org>
Date:   2014-10-31T05:17:45Z

    Merge branch 'index_support' of http://git-wip-us.apache.org/repos/asf/tajo into TAJO-838_3

commit 7a2d8201214cc0f453357c7c935930b8247d537e
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-02T10:59:06Z

    Merge branch 'index_support' of https://git-wip-us.apache.org/repos/asf/tajo into TAJO-838_3

commit 5e36337d455e78ac199164458c32d39c45f296c7
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-02T12:38:18Z

    Merge branch 'TAJO-838_3' of github.com:jihoonson/tajo-2 into TAJO-838_3

commit e9b9b89cdee86c626930d97fe6a4f76051de3531
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-03T02:02:07Z

    Merge branch 'TAJO-838_3' of github.com:jihoonson/tajo-2 into TAJO-838_3

commit 8640170536b33c3870d9968c81bc27fe2a42ebc6
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-03T07:43:35Z

    TAJO-838_3

commit 61255fbba10f52c891eb30602739abf1eeb95c55
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-04T12:37:00Z

    Merge branch 'index_support' of https://git-wip-us.apache.org/repos/asf/tajo into TAJO-838_3
    
    Conflicts:
    	tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java

commit 9883a363fb7b90b5733c87dd800f0ae2c3cae5b1
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-05T10:57:04Z

    TAJO-838_3

commit cf9afcf75b0c090d45df6be686d8bd41b42abd2d
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-06T09:03:00Z

    TAJO-838_3

commit ee4a07b49c8e39ba854896d4e90237c3411de9fe
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-06T15:23:11Z

    TAJO-838_3

commit b99505bf42a34bcf081c5274e46a74e5185aa239
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-07T09:36:31Z

    TAJO-838_3

commit f7747ece5108e480091386c3734dd79c772fdc2d
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-09T09:03:09Z

    TAJO-838_3

commit 477c1d1583512fd77ca25ab3e02467b395bba95b
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-09T09:06:19Z

    Merge branch 'index_support' of https://git-wip-us.apache.org/repos/asf/tajo into TAJO-838_3

commit c12a8e96fd77823b8f828e2160492120da396e3d
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-09T09:19:46Z

    TAJO-838_3

commit 06f5d3e76587e3cfae459648cda20cf02198ffdf
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-09T10:36:33Z

    TAJO-838_3

commit 42d254cd50f1524cf93d4b1d23472966a0a66b61
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-09T11:56:33Z

    TAJO-838_3

commit e483b3923c4a9d201137811bef01a6d22c6e8776
Author: Jihoon Son <ji...@apache.org>
Date:   2014-11-09T14:29:59Z

    TAJO-838_3

----


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20972586
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java ---
    @@ -1855,18 +1877,24 @@ public boolean existIndexByColumn(String databaseName, String tableName, String
     
         try {
           int databaseId = getDatabaseId(databaseName);
    +      int tableId = getTableId(databaseId, databaseName, tableName);
    +      Schema relationSchema = new Schema(getTable(databaseName, tableName).getSchema());
     
           String sql =
    -          "SELECT INDEX_NAME FROM " + TB_INDEXES + " WHERE " + COL_DATABASES_PK + "=? AND COLUMN_NAME=?";
    +          "SELECT " + COL_INDEXES_PK + " FROM " + TB_INDEXES +
    +              " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_PK + "=? AND COLUMN_NAMES=?";
     
           if (LOG.isDebugEnabled()) {
             LOG.debug(sql);
           }
     
    +      String FQUnifiedColumnName = CatalogUtil.buildFQName(databaseName, tableName,
    --- End diff --
    
    Could you check its naming convention?


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-64318012
  
    Hi,
    
    I'm sorry for late review. I'll finish its review by 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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-67725187
  
    I'll finish the review by tonight.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-64321799
  
    No problem.
    Please take it slow. :)


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-67743339
  
    Hi @jihoonson,
    
    This is a great work as well as a huge work. I almost finish the code review. We discussed the code for long time. There still remain feature reviews. I'll test features on a cluster. Then, I'll finish the review 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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-64891072
  
    The query execution time seems to be exchanged. Most of IndexScan results are slower than those of SeqScan.


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

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

    https://github.com/apache/tajo/pull/231#issuecomment-62341166
  
    I'll conduct some performance tests of index and share the result.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-62593817
  
    Hi @jihoonson,
    
    I'm still reviewing the patch. I'll ping you when I finish the review.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-66514869
  
    I'm still reviewing the patch. I expect that the review will be finished tomorrow.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-66248878
  
    @jihoonson 
    This patch needs to be rebased against the latest revision. and Could you take care of this error ?
    ```
    [ERROR] /home/travis/build/apache/tajo/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java:[59,7] org.apache.tajo.catalog.store.HCatalogStore is not abstract and does not override abstract method existIndexesByTable(java.lang.String,java.lang.String) in org.apache.tajo.catalog.store.CatalogStore
    [ERROR] /home/travis/build/apache/tajo/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java:[793,2] method does not override or implement a method from a supertype
    ```


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r22325778
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/AccessPathRewriter.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.plan.rewrite.rules;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.OverridableConf;
    +import org.apache.tajo.SessionVars;
    +import org.apache.tajo.plan.LogicalPlan;
    +import org.apache.tajo.plan.PlanningException;
    +import org.apache.tajo.plan.logical.IndexScanNode;
    +import org.apache.tajo.plan.logical.LogicalNode;
    +import org.apache.tajo.plan.logical.RelationNode;
    +import org.apache.tajo.plan.logical.ScanNode;
    +import org.apache.tajo.plan.rewrite.RewriteRule;
    +import org.apache.tajo.plan.util.PlannerUtil;
    +import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor;
    +
    +import java.util.List;
    +import java.util.Stack;
    +
    +public class AccessPathRewriter implements RewriteRule {
    +  private static final Log LOG = LogFactory.getLog(AccessPathRewriter.class);
    +
    +  private static final String NAME = "Access Path Rewriter";
    +  private Rewriter rewriter = null;
    +
    +  @Override
    +  public String getName() {
    +    return NAME;
    +  }
    +
    +  @Override
    +  public boolean isEligible(OverridableConf conf, LogicalPlan plan) {
    +    if (conf.getBool(SessionVars.INDEX_ENABLED)) {
    +      for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
    +        for (RelationNode relationNode : block.getRelations()) {
    +          List<AccessPathInfo> accessPathInfos = block.getAccessInfos(relationNode);
    +          // If there are any alternative access paths
    +          if (accessPathInfos.size() > 1) {
    +            for (AccessPathInfo accessPathInfo : accessPathInfos) {
    +              if (accessPathInfo.getScanType() == AccessPathInfo.ScanTypeControl.INDEX_SCAN) {
    +                rewriter = new Rewriter(conf);
    --- End diff --
    
    It would be better if Rewrite is a singleton instance and its visitor takes TajoConf. If so, it does not need to create a Rewriter instance for each query.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20977287
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java ---
    @@ -1855,18 +1877,24 @@ public boolean existIndexByColumn(String databaseName, String tableName, String
     
         try {
           int databaseId = getDatabaseId(databaseName);
    +      int tableId = getTableId(databaseId, databaseName, tableName);
    +      Schema relationSchema = new Schema(getTable(databaseName, tableName).getSchema());
     
           String sql =
    -          "SELECT INDEX_NAME FROM " + TB_INDEXES + " WHERE " + COL_DATABASES_PK + "=? AND COLUMN_NAME=?";
    +          "SELECT " + COL_INDEXES_PK + " FROM " + TB_INDEXES +
    +              " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_PK + "=? AND COLUMN_NAMES=?";
     
           if (LOG.isDebugEnabled()) {
             LOG.debug(sql);
           }
     
    +      String FQUnifiedColumnName = CatalogUtil.buildFQName(databaseName, tableName,
    --- End diff --
    
    ok. I'll change.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r21732356
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlan.java ---
    @@ -147,20 +149,33 @@ public String generateUniqueColumnName(EvalNode evalNode) {
         return attachSeqIdToGeneratedColumnName(prefix).toLowerCase();
       }
     
    +  public String generateUniqueColumnName(Expr expr) {
    +    return generateUniqueColumnName(expr, false);
    +  }
    +
       /**
        * It generates an unique column name from Expr. It is usually used for an expression or predicate without
        * a specified name (i.e., alias).
    +   * Here, some expressions require to be identified with their names in the future.
    +   * For example, expressions must be identifiable with their names when getting targets in {@link LogicalPlanner#visitCreateIndex}.
        */
    -  public String generateUniqueColumnName(Expr expr) {
    +  public String generateUniqueColumnName(Expr expr, boolean identifiable) {
         String generatedName;
         if (expr.getType() == OpType.Column) {
           generatedName = ((ColumnReferenceExpr) expr).getCanonicalName();
    +    } else if (identifiable) {
    +      generatedName = generateUniqueIdentifiableColumnName(expr);
         } else { // if a generated column name
           generatedName = attachSeqIdToGeneratedColumnName(getGeneratedPrefixFromExpr(expr));
         }
         return generatedName;
       }
     
    +  private String generateUniqueIdentifiableColumnName(Expr expr) {
    +    IdentifiableNameBuilder nameBuilder = new IdentifiableNameBuilder(expr);
    --- End diff --
    
    Its purpose looks similar to the original generateUniqueColumnName. You may make it due to some bugs of generateUniqueColumnName. I think that it would better if you fix some bug of generateUniqueColumnName instead of making new implementation. 


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r21685588
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/FilterPushDownRule.java ---
    @@ -901,6 +942,49 @@ public LogicalNode visitScan(FilterPushDownContext context, LogicalPlan plan,
         return scanNode;
       }
     
    +  private static class Predicate {
    --- End diff --
    
    Yes, it has a subset feature of EvalNode. 
    Given a conjunctive form of query predicates, we need to disjoint them to find an alternative access path using a proper index; first, we find a proper index with the columns involved in predicates. After that, we estimate the selectivity of those predicates.
    
    As you know, the graph of EvalNodes can represent complex and conjunctive predicates. So, in order to find the index scan path, we need to decompose the EvalNode graph into the several disjoint predicates. 
    
    Currently, the BST index supports only the equality predicate, I used Predicate and SimplePredicate classes as a temporal solution to represent a simple predicate which has a form of 'column = value' or 'value = column'. 
    
    However, we need to devise a better representation for other types of predicates such as range predicate.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20977295
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java ---
    @@ -56,6 +60,7 @@ public void close() throws IOException {
         databases.clear();
         functions.clear();
         indexes.clear();
    +    indexes.clear();
    --- End diff --
    
    Thanks. 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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-67725340
  
    Thanks! 
    I'm going to implement the histogram in Tajo. 
    It will be very helpful if you finish the review this patch. 


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20977336
  
    --- Diff: tajo-common/src/main/java/org/apache/tajo/util/TUtil.java ---
    @@ -258,4 +258,30 @@ public static String getCurrentCodePoint(final int depth) {
         StackTraceElement element = ste[2 + depth];
         return element.getClassName() + ":" + element.getMethodName() + "(" + element.getLineNumber() +")";
       }
    +
    +  /**
    +   * It returns every subsets of the given original set.
    +   *
    +   * @param originalSet an input set
    +   * @param <T> the type of elements of the input set
    +   * @return every subsets of the given original set.
    +   */
    +  public static <T> List<List<T>> powerSet(List<T> originalSet) {
    --- End diff --
    
    Thanks. I forgot that Google always provides everything I need.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-62528842
  
    Thanks for your good suggestion. 
    I changed the title. 


---
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-838: Improve query planner to utilize inde...

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

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


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

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

    https://github.com/apache/tajo/pull/231#issuecomment-62520463
  
    It would be great if you change the pull request title to more descriptive title :)


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-62916666
  
    I'll rebase the patch 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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-63013040
  
    Hyunsik, sorry for late response.
    I forgot to reflect your comment. 
    I'll update my patch.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-66251502
  
    @jinossy 
    Thanks for your comment. I missed it.
    I have updated the patch.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-66604278
  
    I have added more comments for your convenience.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20162566
  
    --- Diff: tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java ---
    @@ -788,14 +788,14 @@ public void createIndex(CatalogProtos.IndexDescProto proto) throws CatalogExcept
       }
     
       @Override
    -  public CatalogProtos.IndexDescProto getIndexByColumn(String databaseName, String tableName, String columnName)
    +  public CatalogProtos.IndexDescProto getIndexByColumns(String databaseName, String tableName, String []columnName)
           throws CatalogException {
         // TODO - not implemented yet
         return null;
    --- End diff --
    
    How about either UnimplementedException or UnsupportedException?


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-64740977
  
    @hyunsik thanks for your detailed review.
    It's very helpful.
    Aforementioned, I'll leave a detailed description for identifiable name, and update the patch after adding more comments.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-68103174
  
    I have added some documents. 
    I'm not familiar with English, so always welcome any comments. :)


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-67810694
  
    While I'm testing, I feel that this work needs some documentation. Could you add its documentation? Of course, I can find some semantic definition and read examples in unit tests. But, it still requires documentation for users and testers. In addition, we may mark it as 'experimental feature' in the documentation.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-66566230
  
    @hyunsik thanks for your review.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r22325621
  
    --- Diff: tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java ---
    @@ -868,4 +865,43 @@ public static KeyValueSet newPhysicalProperties(StoreType type) {
     
         return options;
       }
    +
    +  /**
    +   * Make a unique name by concatenating column names.
    +   * The concatenation is performed in sequence of columns' occurrence in the relation schema.
    +   *
    +   * @param originalSchema original relation schema
    +   * @param columnNames column names which will be unified
    +   * @return unified name
    +   */
    +  public static String getUnifiedSimpleColumnName(Schema originalSchema, String[] columnNames) {
    --- End diff --
    
    How about the case that some column name includes other column names? In this case, it works correctly?


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-62916612
  
    This is a result of performance test.
    * Data: synthetic data
     * Size: 74.4GB
     * # of rows: 357913940
     * DDL
      * create external table gen2 (id int4, sel_0_001 int4, sel_0_01 int4, sel_0_1 int4, sel_1 int4, sel_5 int4, sel_10 int4, sel_20 int4, f1 text, f2 text, f3 text, f4 text, f5 text, f6 text, f7 text, f8 text, f9 text, f10 text, f11 text, pad text) USING CSV WITH ('csvfile.delimiter'=',') LOCATION '/gen/50g_2';
       * The sel_0_001, sel_0_01, sel_0_1, sel_1, sel_5, sel_10, and sel_20 columns have 0.001, 0.01, 0.1, 1, 5, 10, and 20% selectivities, respectively.
    * Query: select id from gen2 where $col = 1;
     * $col: sel_0_001, sel_0_01, sel_0_1, sel_1, sel_5, sel_10, sel_20
    * Result
     * Creation time (sec)
    
    | Index name | Time | 
    |:-------------:|:-------------:| 
    |sel_0_001|159.237|
    |sel_0_01|137.716|
    |sel_0_1|135.772|
    |sel_1|121.511|
    |sel_5|124.585|
    |sel_10|123.593|
    |sel_20|120.621|
    
     * Query execution time (sec)
    
    | $col | Seq scan | Index scan |
    |:-------------:|:-------------:|:-------------:| 
    |sel_0_001|8.445|67.681|
    |sel_0_01|16.306|75.628|
    |sel_0_1|54.602|74.602|
    |sel_1|98.552|78.056|
    |sel_5|104.018|69.755|
    |sel_10|117.401|81.483|
    |sel_20|103.993|71.334|


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-69529509
  
    @hyunsik thanks for your agreement.
    I'll implement remaining things according to the plan.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20968435
  
    --- Diff: tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexMeta.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.catalog;
    +
    +import com.google.common.base.Objects;
    +import com.google.gson.Gson;
    +import com.google.gson.GsonBuilder;
    +import com.google.gson.annotations.Expose;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
    +import org.apache.tajo.util.KeyValueSet;
    +import org.apache.tajo.util.TUtil;
    +
    +import java.util.Arrays;
    +import java.util.Comparator;
    +
    +public class IndexMeta implements Cloneable {
    +  @Expose private String indexName;
    +  @Expose private IndexMethod indexMethod;
    +  @Expose private Path indexPath;
    +  // keySortSpecs are always sorted in the order of the targetRelationSchema
    +  @Expose private SortSpec[] keySortSpecs;
    +  @Expose private boolean isUnique = false;
    +  @Expose private boolean isClustered = false;
    +  @Expose private Schema targetRelationSchema;
    --- End diff --
    
    Does It mean a list of columns to index or a list of all columns of a target relation?


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20977347
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java ---
    @@ -458,39 +474,49 @@ public QueryId updateQuery(QueryContext queryContext, String sql, boolean isJson
       }
     
       private boolean updateQuery(QueryContext queryContext, LogicalNode root) throws IOException {
    -
    +    boolean requireForward = false;
         switch (root.getType()) {
           case CREATE_DATABASE:
             CreateDatabaseNode createDatabase = (CreateDatabaseNode) root;
             createDatabase(queryContext, createDatabase.getDatabaseName(), null, createDatabase.isIfNotExists());
    -        return true;
    +        break;
           case DROP_DATABASE:
             DropDatabaseNode dropDatabaseNode = (DropDatabaseNode) root;
             dropDatabase(queryContext, dropDatabaseNode.getDatabaseName(), dropDatabaseNode.isIfExists());
    -        return true;
    +        break;
           case CREATE_TABLE:
             CreateTableNode createTable = (CreateTableNode) root;
             createTable(queryContext, createTable, createTable.isIfNotExists());
    -        return true;
    +        break;
           case DROP_TABLE:
             DropTableNode dropTable = (DropTableNode) root;
             dropTable(queryContext, dropTable.getTableName(), dropTable.isIfExists(), dropTable.isPurge());
    -        return true;
    +        break;
           case ALTER_TABLESPACE:
             AlterTablespaceNode alterTablespace = (AlterTablespaceNode) root;
             alterTablespace(queryContext, alterTablespace);
    -        return true;
    +        break;
           case ALTER_TABLE:
             AlterTableNode alterTable = (AlterTableNode) root;
             alterTable(queryContext,alterTable);
    -        return true;
    +        break;
           case TRUNCATE_TABLE:
             TruncateTableNode truncateTable = (TruncateTableNode) root;
             truncateTable(queryContext, truncateTable);
    -        return true;
    +        break;
    +      case CREATE_INDEX:
    +        CreateIndexNode createIndexNode = (CreateIndexNode) root;
    +        createIndex(queryContext, createIndexNode);
    +        requireForward = true;
    +        break;
    +      case DROP_INDEX:
    +        DropIndexNode dropIndexNode = (DropIndexNode) root;
    +        dropIndex(queryContext, dropIndexNode);
    +        break;
           default:
             throw new InternalError("updateQuery cannot handle such query: \n" + root.toJson());
         }
    +    return requireForward;
    --- End diff --
    
    I'll add comments.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-68331921
  
    @hyunsik, thanks for your comment.
    Here are my answers.
    
    1. On physical properties: When creating a BST index, the index keys are sorted. The sort specifications in the ```CREATE INDEX``` statement are used to specify the sort order of index keys. Maybe, we can use this feature for performance boost of some other operations such as merge sort or sort-merge join. 
    Since the sort specifications represent the sort order of index keys in a newly created index, I think that users can specify their preferred sort specifications which will be beneficial for future queries. 
    2. On the index maintenance issue: If a base table is changed, every index created on that table must be updated. To handle this problem, we need to introduce a new feature, such as the trigger of database systems. However, since this will induce a huge amount of work, it would be better to proceed in another issue. Until that, we may leave the index maintenance problem to users. IMHO, since this feature is still experimental, it would be enough.
    3. On showing index information: you can see information about the created indexes when executing the ```\d [tablename]``` command. Of course, we need to show more detailed information in future. I'll add this to our documentation, too.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20976038
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java ---
    @@ -350,9 +351,12 @@ public synchronized void startQuery() {
             LOG.warn("Query already started");
             return;
           }
    +      LOG.info(SessionVars.INDEX_ENABLED.keyname() + " : " + queryContext.getBool(SessionVars.INDEX_ENABLED));
           CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
           LogicalPlanner planner = new LogicalPlanner(catalog);
    -      LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
    +      LogicalOptimizer optimizer = new LogicalOptimizer(systemConf, catalog,
    --- End diff --
    
    I think that the role of ```AccessPathRewriterContext``` moved into SessionVars. AccessPathRewriterContext includes a flag ```enableIndex``` and a float variable ```selectivityThreshold```. SessionVars can cover its role. 
    
    Also, AccessPathRewriterContext includes one rewrite rule which is performed according to two variables of AccessPathRewriterContext. For that, it would be better if you change two methods ```isEligible``` and ```rewrite```  ```isEligible(OverridenConf, LogicalPlan)``` and ```rewrite(OverriddenConf, LogicalPlan)```. Because OverriddenConf which is a super class of ```QueryContext``` which includes all variables in SessionVars, two methods will work according to ```enableIndex``` and ```selectivityThreshold``` given by users.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r21750916
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlan.java ---
    @@ -147,20 +149,33 @@ public String generateUniqueColumnName(EvalNode evalNode) {
         return attachSeqIdToGeneratedColumnName(prefix).toLowerCase();
       }
     
    +  public String generateUniqueColumnName(Expr expr) {
    +    return generateUniqueColumnName(expr, false);
    +  }
    +
       /**
        * It generates an unique column name from Expr. It is usually used for an expression or predicate without
        * a specified name (i.e., alias).
    +   * Here, some expressions require to be identified with their names in the future.
    +   * For example, expressions must be identifiable with their names when getting targets in {@link LogicalPlanner#visitCreateIndex}.
        */
    -  public String generateUniqueColumnName(Expr expr) {
    +  public String generateUniqueColumnName(Expr expr, boolean identifiable) {
         String generatedName;
         if (expr.getType() == OpType.Column) {
           generatedName = ((ColumnReferenceExpr) expr).getCanonicalName();
    +    } else if (identifiable) {
    +      generatedName = generateUniqueIdentifiableColumnName(expr);
         } else { // if a generated column name
           generatedName = attachSeqIdToGeneratedColumnName(getGeneratedPrefixFromExpr(expr));
         }
         return generatedName;
       }
     
    +  private String generateUniqueIdentifiableColumnName(Expr expr) {
    +    IdentifiableNameBuilder nameBuilder = new IdentifiableNameBuilder(expr);
    --- End diff --
    
    As you commented, the identifiable name is used to resolve a problem due to a bug of generateUniqueColumnName(). But now, that bug seems to be fixed. So, I removed related 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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-69185433
  
    @hyunsik thanks for the review!
    
    There still remains many things for efficient query processing with indexes. Here are the significant issues.
    1. Printing appropriate messages after creating an index. If the index is successfully created, cli should show a success message. Otherwise, it should show a failure message. 
    2. Index metadata backup and restore. In order to support the index restore without recreation, we need to provide a new DDL statement, ```CREATE INDEX [name] ON [path]```.
    3. Index support for partitioned tables.
    4. New index structure.
    5. Other experimental issues, such as prefetching indexes, for the performance improvement.
    
    IMHO, after resolving the issues 1 and 2, it would be sufficient to be merged into the master branch.
    What do you think about 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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-69483361
  
    @jihoonson ,
    
    I agree with your plan. Even through this feature is still experimental, it seems to be ready to merge this feature to master branch due to maintenance cost. It also works well for CSV files.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20972931
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java ---
    @@ -458,39 +474,49 @@ public QueryId updateQuery(QueryContext queryContext, String sql, boolean isJson
       }
     
       private boolean updateQuery(QueryContext queryContext, LogicalNode root) throws IOException {
    -
    +    boolean requireForward = false;
         switch (root.getType()) {
           case CREATE_DATABASE:
             CreateDatabaseNode createDatabase = (CreateDatabaseNode) root;
             createDatabase(queryContext, createDatabase.getDatabaseName(), null, createDatabase.isIfNotExists());
    -        return true;
    +        break;
           case DROP_DATABASE:
             DropDatabaseNode dropDatabaseNode = (DropDatabaseNode) root;
             dropDatabase(queryContext, dropDatabaseNode.getDatabaseName(), dropDatabaseNode.isIfExists());
    -        return true;
    +        break;
           case CREATE_TABLE:
             CreateTableNode createTable = (CreateTableNode) root;
             createTable(queryContext, createTable, createTable.isIfNotExists());
    -        return true;
    +        break;
           case DROP_TABLE:
             DropTableNode dropTable = (DropTableNode) root;
             dropTable(queryContext, dropTable.getTableName(), dropTable.isIfExists(), dropTable.isPurge());
    -        return true;
    +        break;
           case ALTER_TABLESPACE:
             AlterTablespaceNode alterTablespace = (AlterTablespaceNode) root;
             alterTablespace(queryContext, alterTablespace);
    -        return true;
    +        break;
           case ALTER_TABLE:
             AlterTableNode alterTable = (AlterTableNode) root;
             alterTable(queryContext,alterTable);
    -        return true;
    +        break;
           case TRUNCATE_TABLE:
             TruncateTableNode truncateTable = (TruncateTableNode) root;
             truncateTable(queryContext, truncateTable);
    -        return true;
    +        break;
    +      case CREATE_INDEX:
    +        CreateIndexNode createIndexNode = (CreateIndexNode) root;
    +        createIndex(queryContext, createIndexNode);
    +        requireForward = true;
    +        break;
    +      case DROP_INDEX:
    +        DropIndexNode dropIndexNode = (DropIndexNode) root;
    +        dropIndex(queryContext, dropIndexNode);
    +        break;
           default:
             throw new InternalError("updateQuery cannot handle such query: \n" + root.toJson());
         }
    +    return requireForward;
    --- End diff --
    
    The behavior of ```updateQuery``` seems to be changed. Could you add comment about updateQuery metnod?


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-69150753
  
    Updated against the master branch.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-64737374
  
    I leave comments.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-64948522
  
    @hyunsik, you are right.
    I fixed the result table.
    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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20972628
  
    --- Diff: tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java ---
    @@ -56,6 +60,7 @@ public void close() throws IOException {
         databases.clear();
         functions.clear();
         indexes.clear();
    +    indexes.clear();
    --- End diff --
    
    The variable ```indexes``` is cleared twice.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20977276
  
    --- Diff: tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexMeta.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.catalog;
    +
    +import com.google.common.base.Objects;
    +import com.google.gson.Gson;
    +import com.google.gson.GsonBuilder;
    +import com.google.gson.annotations.Expose;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
    +import org.apache.tajo.util.KeyValueSet;
    +import org.apache.tajo.util.TUtil;
    +
    +import java.util.Arrays;
    +import java.util.Comparator;
    +
    +public class IndexMeta implements Cloneable {
    +  @Expose private String indexName;
    +  @Expose private IndexMethod indexMethod;
    +  @Expose private Path indexPath;
    +  // keySortSpecs are always sorted in the order of the targetRelationSchema
    +  @Expose private SortSpec[] keySortSpecs;
    +  @Expose private boolean isUnique = false;
    +  @Expose private boolean isClustered = false;
    +  @Expose private Schema targetRelationSchema;
    --- End diff --
    
    It means a list of all columns of a target relation. I'll add some comments for 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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-64732705
  
    Since the jira issue number (Tajo-838) in the github issue title includes lower case, this github ticket is not linked to the jira. So, could you change the jira issue number to upper case?


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r21676976
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/FilterPushDownRule.java ---
    @@ -901,6 +942,49 @@ public LogicalNode visitScan(FilterPushDownContext context, LogicalPlan plan,
         return scanNode;
       }
     
    +  private static class Predicate {
    --- End diff --
    
    Predicate seems to have a subset feature of EvalNode. Could you explain some background to add 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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20977379
  
    --- Diff: tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java ---
    @@ -350,9 +351,12 @@ public synchronized void startQuery() {
             LOG.warn("Query already started");
             return;
           }
    +      LOG.info(SessionVars.INDEX_ENABLED.keyname() + " : " + queryContext.getBool(SessionVars.INDEX_ENABLED));
           CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
           LogicalPlanner planner = new LogicalPlanner(catalog);
    -      LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
    +      LogicalOptimizer optimizer = new LogicalOptimizer(systemConf, catalog,
    --- End diff --
    
    Thanks for the variable comment. I'll refactor this part according to your comment.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-67812150
  
    You are right. I'll add some documents.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20976101
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/NamedExprsManager.java ---
    @@ -193,11 +193,11 @@ public String addExpr(Expr expr, String alias) throws PlanningException {
        * Adds an expression and returns a reference name.
        * If an alias is given, it specifies the alias as an reference name.
        */
    -  public String addNamedExpr(NamedExpr namedExpr) throws PlanningException {
    +  public String addNamedExpr(NamedExpr namedExpr, boolean identifiable) throws PlanningException {
         if (namedExpr.hasAlias()) {
           return addExpr(namedExpr.getExpr(), namedExpr.getAlias());
         } else {
    -      return addExpr(namedExpr.getExpr());
    +      return addExpr(namedExpr.getExpr(), identifiable);
    --- End diff --
    
    What is identifiable name? It is hard to imagine its purpose. Could you elaborate its purpose and add comments to the changed interfaces?


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-69170263
  
    +1
    It seems to be ready to be committed to index_support branch. Even through the test results in two failures, I don't think that they are not related to your patch. They are related to some blocker issues TAJO-1282 and TAJO-1278. They will be resolved in other Jiras.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-66231475
  
    Would anyone review the patch?


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20972854
  
    --- Diff: tajo-common/src/main/java/org/apache/tajo/util/TUtil.java ---
    @@ -258,4 +258,30 @@ public static String getCurrentCodePoint(final int depth) {
         StackTraceElement element = ste[2 + depth];
         return element.getClassName() + ":" + element.getMethodName() + "(" + element.getLineNumber() +")";
       }
    +
    +  /**
    +   * It returns every subsets of the given original set.
    +   *
    +   * @param originalSet an input set
    +   * @param <T> the type of elements of the input set
    +   * @return every subsets of the given original set.
    +   */
    +  public static <T> List<List<T>> powerSet(List<T> originalSet) {
    --- End diff --
    
    It looks good. BTW, you also can use ```Sets.powerSet()``` method.
    
    http://docs.guava-libraries.googlecode.com/git/javadoc/com/google/common/collect/Sets.html#powerSet(java.util.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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-68293902
  
    It's excellent. It's works very well. It's performance gain was larger than I expected. I definitely think that this feature is very promising. Probably, this feature may differentiate Tajo from other competitive systems.
    
    The objective of following questions is to make this feature more solid and more consistent.
    
    While I'm testing features on a local cluster, I get some questions.
    
    In some tests, CREATE INDEX statement includes physical properties as the following example:
    ```
    create index l_orderkey_idx on lineitem (l_orderkey asc null first);
    ```
    How does a specified sort order affects index creation? I tested it with ```asc``` as well as ```desc```. But, it works well in both cases. Is it just a hint?
    
    In addition, such a physical property is fragile due to following reasons:
     * It is hard for users to ensure the table properties.
     * INSERT (OVERWRITE) INTO AS SELECT can change the physical properties.
    
    How do you think about them? After I get your answer, I'll give additional questions.
    
    Also, I have feature wish.
     * There is no way to show already created indexes and their information.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-68510211
  
    Updated against the master branch.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r22339002
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/AccessPathRewriter.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.plan.rewrite.rules;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.OverridableConf;
    +import org.apache.tajo.SessionVars;
    +import org.apache.tajo.plan.LogicalPlan;
    +import org.apache.tajo.plan.PlanningException;
    +import org.apache.tajo.plan.logical.IndexScanNode;
    +import org.apache.tajo.plan.logical.LogicalNode;
    +import org.apache.tajo.plan.logical.RelationNode;
    +import org.apache.tajo.plan.logical.ScanNode;
    +import org.apache.tajo.plan.rewrite.RewriteRule;
    +import org.apache.tajo.plan.util.PlannerUtil;
    +import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor;
    +
    +import java.util.List;
    +import java.util.Stack;
    +
    +public class AccessPathRewriter implements RewriteRule {
    +  private static final Log LOG = LogFactory.getLog(AccessPathRewriter.class);
    +
    +  private static final String NAME = "Access Path Rewriter";
    +  private Rewriter rewriter = null;
    +
    +  @Override
    +  public String getName() {
    +    return NAME;
    +  }
    +
    +  @Override
    +  public boolean isEligible(OverridableConf conf, LogicalPlan plan) {
    +    if (conf.getBool(SessionVars.INDEX_ENABLED)) {
    +      for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
    +        for (RelationNode relationNode : block.getRelations()) {
    +          List<AccessPathInfo> accessPathInfos = block.getAccessInfos(relationNode);
    +          // If there are any alternative access paths
    +          if (accessPathInfos.size() > 1) {
    +            for (AccessPathInfo accessPathInfo : accessPathInfos) {
    +              if (accessPathInfo.getScanType() == AccessPathInfo.ScanTypeControl.INDEX_SCAN) {
    +                rewriter = new Rewriter(conf);
    --- End diff --
    
    Thank you for the good suggestion.
    I'll follow your suggestion.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r20977436
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/NamedExprsManager.java ---
    @@ -193,11 +193,11 @@ public String addExpr(Expr expr, String alias) throws PlanningException {
        * Adds an expression and returns a reference name.
        * If an alias is given, it specifies the alias as an reference name.
        */
    -  public String addNamedExpr(NamedExpr namedExpr) throws PlanningException {
    +  public String addNamedExpr(NamedExpr namedExpr, boolean identifiable) throws PlanningException {
         if (namedExpr.hasAlias()) {
           return addExpr(namedExpr.getExpr(), namedExpr.getAlias());
         } else {
    -      return addExpr(namedExpr.getExpr());
    +      return addExpr(namedExpr.getExpr(), identifiable);
    --- End diff --
    
    I'll leave a detailed description for the identifiable name, tomorrow.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-66787842
  
    @hyunsik thanks for your comment.
    I updated the patch. 


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

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

    https://github.com/apache/tajo/pull/231#issuecomment-62341101
  
    I fixed some bugs in the catalog after testing in a real cluster.


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#discussion_r22339025
  
    --- Diff: tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java ---
    @@ -868,4 +865,43 @@ public static KeyValueSet newPhysicalProperties(StoreType type) {
     
         return options;
       }
    +
    +  /**
    +   * Make a unique name by concatenating column names.
    +   * The concatenation is performed in sequence of columns' occurrence in the relation schema.
    +   *
    +   * @param originalSchema original relation schema
    +   * @param columnNames column names which will be unified
    +   * @return unified name
    +   */
    +  public static String getUnifiedSimpleColumnName(Schema originalSchema, String[] columnNames) {
    --- End diff --
    
    The value ```originalSchema``` is a schema of a relation. I think that we should guarantee that every column name is unique in a relation. 


---
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-838: Improve query planner to utilize inde...

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

    https://github.com/apache/tajo/pull/231#issuecomment-67758735
  
    @hyunsik, thanks for your review.
    For your convenience, I would like to share my test environment.
    First of all, you need to delete the catalog database because the schema of the index table is changed.
    Please back up your catalog before deletion.
    When you create an index successfully, please don't forget to configure the session variable. You must set INDEX_ENABLED as true to enable the index scan feature.
    If you have any questions, please feel free to ask me.
    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.
---