You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by sv71294 <gi...@git.apache.org> on 2018/05/22 12:12:25 UTC

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

GitHub user sv71294 opened a pull request:

    https://github.com/apache/carbondata/pull/2334

    [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp greaterthan expression and OR filter Expression issue

    **Problem 1 [CARBONDATA-2515]:** OR expression filters not handled in PrestoFilterUtil
    
    **Problem 2 [CARBONDATA-2516]:** Full blocklet scanning for Timestamp Greater than expression as it is not generating in PrestoFilterUtil.
    
    **Solution**: added functionality for OR expression in PrestoFilterUtil and relevant code fixes for the timestamp issue.

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

    $ git pull https://github.com/sv71294/carbondata carbon-presto-filterExpression-fixes

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

    https://github.com/apache/carbondata/pull/2334.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 #2334
    
----
commit 355f41cf78ee337c224ff44c0ebe0b92c1672b16
Author: sv71294 <sv...@...>
Date:   2018-05-22T11:35:18Z

    carbon presto integraion timestamp greaterthan expression and OR filter expression fixed

----


---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r197669862
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -171,134 +170,89 @@ else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.get
        * @return
        */
       static Expression parseFilterExpression(TupleDomain<ColumnHandle> originalConstraint) {
    -    ImmutableList.Builder<Expression> filters = ImmutableList.builder();
     
         Domain domain;
    +    Expression finalFilters = null;
    --- End diff --
    
    Can you provide comments to explain the below Expression?  and why colExpression need put "final"?
    
    Expression finalFilters = null;
    final Expression colExpression = new ColumnExpression(cdch.getColumnName(), coltype);
    Expression colValueExpression = null;
    Expression rangeExpression = null;


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    add to whitelist


---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r197722635
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -171,134 +170,89 @@ else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.get
        * @return
        */
       static Expression parseFilterExpression(TupleDomain<ColumnHandle> originalConstraint) {
    -    ImmutableList.Builder<Expression> filters = ImmutableList.builder();
     
         Domain domain;
    +    Expression finalFilters = null;
    --- End diff --
    
    sure, adding comments in code


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Please review it now


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    @chenliang613 issue is not about the result, in current master filter is getting handled by presto as it is not generated on connector layer, now with this PR it will be generated at connector layer and passed to core for filter blocklet scanning which will reduce the no of rows passed to presto with filter expression else presto will need to filter the row from universal set.
    
    You can also refer to JIRA ISSUE CARBONDATA-2516 for clarity.


---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r190104064
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -110,28 +88,30 @@ else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.get
         for (ColumnHandle columnHandle : originalConstraint.getDomains().get().keySet()) {
           CarbondataColumnHandle carbondataColumnHandle = (CarbondataColumnHandle) columnHandle;
           List<ColumnSchema> partitionedColumnSchema = columnSchemas.stream().filter(
    -          columnSchema -> carbondataColumnHandle.getColumnName().equals(columnSchema.getColumnName())).collect(toList());
    -      if(partitionedColumnSchema.size() != 0) {
    +              columnSchema -> carbondataColumnHandle.getColumnName().equals(columnSchema.getColumnName())).collect(toList());
    +      if (partitionedColumnSchema.size() != 0) {
             filter.addAll(createPartitionFilters(originalConstraint, carbondataColumnHandle));
           }
         }
         return filter;
       }
     
    -  /** Returns list of partition key and values using domain constraints
    +  /**
    +   * Returns list of partition key and values using domain constraints
    +   *
        * @param originalConstraint
        * @param carbonDataColumnHandle
        */
       private static List<String> createPartitionFilters(TupleDomain<ColumnHandle> originalConstraint,
    -      CarbondataColumnHandle carbonDataColumnHandle) {
    +                                                     CarbondataColumnHandle carbonDataColumnHandle) {
    --- End diff --
    
    Why change this one?


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    LGTM


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Can one of the admins verify this patch?


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/5286/



---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r190103764
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -17,51 +17,28 @@
     
     package org.apache.carbondata.presto;
     
    -import java.math.BigDecimal;
    -import java.math.BigInteger;
    -import java.sql.Timestamp;
    -import java.util.ArrayList;
    -import java.util.Calendar;
    -import java.util.Date;
    -import java.util.HashMap;
    -import java.util.List;
    -import java.util.Map;
    -
    +import com.facebook.presto.spi.ColumnHandle;
    +import com.facebook.presto.spi.PrestoException;
    +import com.facebook.presto.spi.predicate.Domain;
    +import com.facebook.presto.spi.predicate.Range;
    +import com.facebook.presto.spi.predicate.TupleDomain;
    +import com.facebook.presto.spi.type.*;
    +import io.airlift.slice.Slice;
     import org.apache.carbondata.core.metadata.datatype.DataType;
     import org.apache.carbondata.core.metadata.datatype.DataTypes;
     import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
     import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
     import org.apache.carbondata.core.scan.expression.ColumnExpression;
     import org.apache.carbondata.core.scan.expression.Expression;
     import org.apache.carbondata.core.scan.expression.LiteralExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.GreaterThanEqualToExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.GreaterThanExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.InExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.LessThanEqualToExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
    +import org.apache.carbondata.core.scan.expression.conditional.*;
     import org.apache.carbondata.core.scan.expression.logical.AndExpression;
     import org.apache.carbondata.core.scan.expression.logical.OrExpression;
     
    -import com.facebook.presto.spi.ColumnHandle;
    -import com.facebook.presto.spi.PrestoException;
    -import com.facebook.presto.spi.predicate.Domain;
    -import com.facebook.presto.spi.predicate.Range;
    -import com.facebook.presto.spi.predicate.TupleDomain;
    -import com.facebook.presto.spi.type.BigintType;
    -import com.facebook.presto.spi.type.BooleanType;
    -import com.facebook.presto.spi.type.DateType;
    -import com.facebook.presto.spi.type.DecimalType;
    -import com.facebook.presto.spi.type.Decimals;
    -import com.facebook.presto.spi.type.DoubleType;
    -import com.facebook.presto.spi.type.IntegerType;
    -import com.facebook.presto.spi.type.SmallintType;
    -import com.facebook.presto.spi.type.TimestampType;
    -import com.facebook.presto.spi.type.Type;
    -import com.facebook.presto.spi.type.VarcharType;
    -import com.google.common.collect.ImmutableList;
    -import io.airlift.slice.Slice;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.sql.Timestamp;
    +import java.util.*;
    --- End diff --
    
    Please keep the detail class, not *


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/6451/



---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r190103900
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -17,51 +17,28 @@
     
     package org.apache.carbondata.presto;
     
    -import java.math.BigDecimal;
    -import java.math.BigInteger;
    -import java.sql.Timestamp;
    -import java.util.ArrayList;
    -import java.util.Calendar;
    -import java.util.Date;
    -import java.util.HashMap;
    -import java.util.List;
    -import java.util.Map;
    -
    +import com.facebook.presto.spi.ColumnHandle;
    +import com.facebook.presto.spi.PrestoException;
    +import com.facebook.presto.spi.predicate.Domain;
    +import com.facebook.presto.spi.predicate.Range;
    +import com.facebook.presto.spi.predicate.TupleDomain;
    +import com.facebook.presto.spi.type.*;
    +import io.airlift.slice.Slice;
     import org.apache.carbondata.core.metadata.datatype.DataType;
     import org.apache.carbondata.core.metadata.datatype.DataTypes;
     import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
     import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
     import org.apache.carbondata.core.scan.expression.ColumnExpression;
     import org.apache.carbondata.core.scan.expression.Expression;
     import org.apache.carbondata.core.scan.expression.LiteralExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.GreaterThanEqualToExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.GreaterThanExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.InExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.LessThanEqualToExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression;
    -import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
    +import org.apache.carbondata.core.scan.expression.conditional.*;
     import org.apache.carbondata.core.scan.expression.logical.AndExpression;
     import org.apache.carbondata.core.scan.expression.logical.OrExpression;
     
    -import com.facebook.presto.spi.ColumnHandle;
    -import com.facebook.presto.spi.PrestoException;
    -import com.facebook.presto.spi.predicate.Domain;
    -import com.facebook.presto.spi.predicate.Range;
    -import com.facebook.presto.spi.predicate.TupleDomain;
    -import com.facebook.presto.spi.type.BigintType;
    -import com.facebook.presto.spi.type.BooleanType;
    -import com.facebook.presto.spi.type.DateType;
    -import com.facebook.presto.spi.type.DecimalType;
    -import com.facebook.presto.spi.type.Decimals;
    -import com.facebook.presto.spi.type.DoubleType;
    -import com.facebook.presto.spi.type.IntegerType;
    -import com.facebook.presto.spi.type.SmallintType;
    -import com.facebook.presto.spi.type.TimestampType;
    -import com.facebook.presto.spi.type.Type;
    -import com.facebook.presto.spi.type.VarcharType;
    -import com.google.common.collect.ImmutableList;
    -import io.airlift.slice.Slice;
    +import java.math.BigDecimal;
    --- End diff --
    
    please optimize the order of class, unify with others


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/5300/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    @chenliang613 please check this image, condition logic was missing
    ![screenshot_20180623-134658](https://user-images.githubusercontent.com/3933895/41807395-3e23bfbe-76ec-11e8-9b50-a2aac2645b0f.png)



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Can one of the admins verify this patch?


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    retest this please


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    retest this please


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    retest this please


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/5303/



---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r190104019
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -110,28 +88,30 @@ else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.get
         for (ColumnHandle columnHandle : originalConstraint.getDomains().get().keySet()) {
           CarbondataColumnHandle carbondataColumnHandle = (CarbondataColumnHandle) columnHandle;
           List<ColumnSchema> partitionedColumnSchema = columnSchemas.stream().filter(
    -          columnSchema -> carbondataColumnHandle.getColumnName().equals(columnSchema.getColumnName())).collect(toList());
    -      if(partitionedColumnSchema.size() != 0) {
    +              columnSchema -> carbondataColumnHandle.getColumnName().equals(columnSchema.getColumnName())).collect(toList());
    --- End diff --
    
    Why change this one?


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    retest this please


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/5282/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/5391/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    @sv71294   
    i can also run successfully at current master with the below test case, so not sure this test case can cover this issue which be fixed in the PR.
    ```
    test("test the Timestamp greaterthan expression"){
        val actualResult: List[Map[String, Any]] = PrestoServer.executeQuery("SELECT DOB FROM TESTDB.TESTTABLE" +
                                                                             " WHERE DOB > timestamp '2016-01-01 00:00:00.0' order by DOB")
        val expectedResult: List[Map[String, Any]] = List(
          Map("DOB" -> new Timestamp(new java.util.Date(2016-1900,1-1,14,15,7,9).getTime)),
          Map("DOB" -> new Timestamp(new java.util.Date(2016-1900,4-1,14,15,0,9).getTime)))
        assert(actualResult.equals(expectedResult))
      }
    ```


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Please optimize the code style of this PR


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/6472/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/4983/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/6469/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Can one of the admins verify this patch?


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    @chenliang613 I have added the required test case for greater than expresion, please check it now


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    @chenliang613 adding the test case to cover timestamp greater than expression issue


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    @chenliang613 I have added the comments in the code, please review it now.


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    retest this please


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/6146/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/5358/



---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r190103982
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -92,14 +69,15 @@ private static DataType Spi2CarbondataTypeMapper(CarbondataColumnHandle carbonda
         else if (colType == DateType.DATE) return DataTypes.DATE;
         else if (colType == TimestampType.TIMESTAMP) return DataTypes.TIMESTAMP;
         else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.getPrecision(),
    -        carbondataColumnHandle.getScale()))) return DataTypes
    -        .createDecimalType(carbondataColumnHandle.getPrecision(),
    -            carbondataColumnHandle.getScale());
    +            carbondataColumnHandle.getScale()))) return DataTypes
    --- End diff --
    
    Why change this one?


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/6455/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/6526/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/5392/



---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by sv71294 <gi...@git.apache.org>.
Github user sv71294 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    Retest this please


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    @sv71294  thank you for your contribution.
    In this pr, i didn't find the test case to cover "Timestamp greaterthan expression"


---

[GitHub] carbondata issue #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Timestamp g...

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on the issue:

    https://github.com/apache/carbondata/pull/2334
  
    For "[CARBONDATA-2515] OR filter Expression issue" , verified.
    ```
    presto:default> select * from carbon_table where ID='No.11' and country='country0' and population > 100 or population < 64;
     id | country | city | population
    ----+---------+------+------------
    (0 rows)
    ```
    The PR's result as below : 
    ```
    presto:default> select * from carbon_table where population > 100 or population < 64;
        id     | country  |  city  | population
    -----------+----------+--------+------------
     No.0      | country0 | city0  |        200
     No.0      | country0 | city0  |          0
     No.0      | country0 | city0  |          0
     No.0      | country0 | city10 |         60
     No.0      | country0 | city12 |        112
     No.0      | country0 | city12 |        112
     No.0      | country0 | city14 |        264
     No.0      | country0 | city14 |        264
     No.0      | country0 | city14 |        164
     No.0      | country0 | city16 |         16
    ```
    



---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r197678803
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -171,134 +170,89 @@ else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.get
        * @return
        */
       static Expression parseFilterExpression(TupleDomain<ColumnHandle> originalConstraint) {
    -    ImmutableList.Builder<Expression> filters = ImmutableList.builder();
     
         Domain domain;
    +    Expression finalFilters = null;
    --- End diff --
    
    1. finalFilters => this one is the final expression for the table,  this variable is returned by the method after combining all the column filters (colValueExpression).
    2. colExpression => we have used final to make sure that value of this variable doesn't get change during the execution of iteration for respective column.
    3. colValueExpression => this is the combination of multiple rangeExpression for a single column in case of multiple filters on single column else this is equal to rangeExpression
    4.rangeExpression => this is generated for each range of column i.e. lessThan, greaterThan, there can be multiple ranges for a single column.


---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334#discussion_r197721852
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -171,134 +170,89 @@ else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.get
        * @return
        */
       static Expression parseFilterExpression(TupleDomain<ColumnHandle> originalConstraint) {
    -    ImmutableList.Builder<Expression> filters = ImmutableList.builder();
     
         Domain domain;
    +    Expression finalFilters = null;
    --- End diff --
    
    1. I mean, please put these detail explanation inside the code in your pr
    2. colExpression, just column's name and data type, why may change ? 



---

[GitHub] carbondata pull request #2334: [CARBONDATA-2515][CARBONDATA-2516] fixed Time...

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

    https://github.com/apache/carbondata/pull/2334


---