You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by anubhav100 <gi...@git.apache.org> on 2017/08/14 17:07:52 UTC

[GitHub] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

GitHub user anubhav100 opened a pull request:

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

    [CARBONDATA-1347] Implemented Columnar Reading Of Data For Presto

    This Pr Include Following Features :
    
     - implemented columanr based iterator and executor for reading columns instead of rows
     - create a new improved dictionarydecoder with scala
     -added the lazy blocks
     - added the stream readers in presto
     - removed the redundant execution of parsing filters using map
    -removed the useless code and redudant intilizations
     -fixed the following bugs
            - Wrong values of decimal types in tpch queries
            - class cast exception in intermediate file merger
            - refactor the carbon data comprator
             
    


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

    $ git pull https://github.com/anubhav100/incubator-carbondata feature/CARBONDATA-1347

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

    https://github.com/apache/carbondata/pull/1257.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 #1257
    
----
commit 8ebe591a19fe40e438cfc35fcd91a9b4d1ba8552
Author: anubhav100 <an...@knoldus.in>
Date:   2017-08-10T09:42:22Z

    wip for implemented columnar reading of data for presto

commit e50ef7e1e4f0ca215ed9f6b0b6095fb6d037e5e6
Author: anubhav100 <an...@knoldus.in>
Date:   2017-08-14T09:31:58Z

    corrected the decimal type

commit b5ce0fe7b56b27750b6d0392eff710331a2fedb4
Author: anubhav100 <an...@knoldus.in>
Date:   2017-08-14T13:05:37Z

    resolved minor bugs in core

----


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/3418/



---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133407524
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/constants/CarbonCommonConstants.java ---
    @@ -0,0 +1,1319 @@
    +/*
    --- End diff --
    
    done


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    @chenliang613 i have update the pr you can have a look


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134779856
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java ---
    @@ -73,27 +60,23 @@ public CarbondataRecordSetProvider(CarbondataConnectorId connectorId, CarbonTabl
     
         CarbondataSplit carbondataSplit =
             checkType(split, CarbondataSplit.class, "split is not class CarbondataSplit");
    -    checkArgument(carbondataSplit.getConnectorId().equals(connectorId), "split is not for this connector");
    +    checkArgument(carbondataSplit.getConnectorId().equals(connectorId),
    +        "split is not for this connector");
     
    -    StringBuffer targetColsBuffer = new StringBuffer();
         String targetCols = "";
         // Convert all columns handles
         ImmutableList.Builder<CarbondataColumnHandle> handles = ImmutableList.builder();
         for (ColumnHandle handle : columns) {
           handles.add(checkType(handle, CarbondataColumnHandle.class, "handle"));
    -      targetColsBuffer.append(((CarbondataColumnHandle) handle).getColumnName()).append(",");
    +      targetCols += ((CarbondataColumnHandle) handle).getColumnName() + ",";
    --- End diff --
    
    Actually last commit was not rebased with my previous one I will  correct it again


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133375740
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/AbstractAggregatedMemoryContext.java ---
    @@ -0,0 +1,37 @@
    +/*
    + * Licensed 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
    + *
    --- End diff --
    
    The license header is wrong.


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/225/



---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134219468
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/LocalMemoryContext.java ---
    @@ -0,0 +1,43 @@
    +/*
    + * 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.carbondata.presto.memory;
    +
    +import static com.google.common.base.MoreObjects.toStringHelper;
    +import static java.util.Objects.requireNonNull;
    +
    +public class LocalMemoryContext {
    --- End diff --
    
    done


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

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


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

[GitHub] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

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


---

[GitHub] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

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


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

[GitHub] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133904450
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/AbstractAggregatedMemoryContext.java ---
    @@ -0,0 +1,40 @@
    +/*
    + * 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.carbondata.presto.memory;
    +
    +
    +abstract class AbstractAggregatedMemoryContext
    +{
    +  // This class should remain exactly the same as AbstractAggregatedMemoryContext in com.facebook.presto.memory
    +
    +  // AbstractMemoryContext class is only necessary because we need implementations that bridge
    +  // AggregatedMemoryContext with the existing memory tracking APIs in XxxxContext. Once they
    +  // are refactored, there will be only one implementation of this abstract class, and this class
    +  // can be removed.
    +
    +  protected abstract void updateBytes(long bytes);
    +
    +  public AggregatedMemoryContext newAggregatedMemoryContext()
    +  {
    --- End diff --
    
    please correct the code format


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134780421
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/constants/PrestoCommonConstants.java ---
    @@ -0,0 +1,29 @@
    +/*
    + * 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.carbondata.presto.constants;
    +
    +/**
    + * Constants for presto functionality
    + */
    --- End diff --
    
    Ok


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

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


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

[GitHub] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134694501
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java ---
    @@ -79,30 +81,27 @@ private Expression parseConstraint2Expression(TupleDomain<ColumnHandle> constrai
       @Override public RecordCursor cursor() {
         List<TableBlockInfo> tableBlockInfoList = new ArrayList<TableBlockInfo>();
     
    -    tableBlockInfoList.add(new TableBlockInfo(split.getLocalInputSplit().getPath().toString(),
    +    tableBlockInfoList.add(new TableBlockInfo(split.getLocalInputSplit().getPath(),
             split.getLocalInputSplit().getStart(), split.getLocalInputSplit().getSegmentId(),
             split.getLocalInputSplit().getLocations().toArray(new String[0]),
             split.getLocalInputSplit().getLength(), new BlockletInfos(),
             //blockletInfos,
             ColumnarFormatVersion.valueOf(split.getLocalInputSplit().getVersion()), null));
    -    queryModel.setTableBlockInfos(tableBlockInfoList);
     
    +    queryModel.setTableBlockInfos(tableBlockInfoList);
         queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
     
    -    try {
    +    CarbonProperties.getInstance().addProperty("carbon.detail.batch.size",
    --- End diff --
    
    I can't find where code use this property(carbon.detail.batch.size)? don't know need to add property at here ?


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133902762
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataFilterUtil.java ---
    @@ -0,0 +1,233 @@
    +/*
    + * 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
    + */
    +
    +package org.apache.carbondata.presto;
    +
    +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 java.util.stream.Collectors;
    +
    +import org.apache.carbondata.core.metadata.datatype.DataType;
    +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.logical.AndExpression;
    +import org.apache.carbondata.core.scan.expression.logical.OrExpression;
    +
    +import com.facebook.presto.spi.ColumnHandle;
    +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.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 static com.google.common.base.Preconditions.checkArgument;
    +
    +public class CarbondataFilterUtil {
    --- End diff --
    
    1. please provide the detail explain about class :  add each function for which purpose ?
    2.Suggest changing the class name from  CarbondataFilterUtil to PrestoFilterUtil


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/238/



---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/197/



---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    @chenliang613 please review this pr it contains all the changes


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133375914
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/LocalMemoryContext.java ---
    @@ -0,0 +1,40 @@
    +/*
    + * Licensed 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.
    --- End diff --
    
    The license header is wrong.


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    Ok to test


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134712614
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java ---
    @@ -79,30 +81,27 @@ private Expression parseConstraint2Expression(TupleDomain<ColumnHandle> constrai
       @Override public RecordCursor cursor() {
         List<TableBlockInfo> tableBlockInfoList = new ArrayList<TableBlockInfo>();
     
    -    tableBlockInfoList.add(new TableBlockInfo(split.getLocalInputSplit().getPath().toString(),
    +    tableBlockInfoList.add(new TableBlockInfo(split.getLocalInputSplit().getPath(),
             split.getLocalInputSplit().getStart(), split.getLocalInputSplit().getSegmentId(),
             split.getLocalInputSplit().getLocations().toArray(new String[0]),
             split.getLocalInputSplit().getLength(), new BlockletInfos(),
             //blockletInfos,
             ColumnarFormatVersion.valueOf(split.getLocalInputSplit().getVersion()), null));
    -    queryModel.setTableBlockInfos(tableBlockInfoList);
     
    +    queryModel.setTableBlockInfos(tableBlockInfoList);
         queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
     
    -    try {
    +    CarbonProperties.getInstance().addProperty("carbon.detail.batch.size",
    --- End diff --
    
    Actually in DataBlockIteratoImpl we are using this property to create the batches, we are setting this property here to ensure that the batchsize of Presto and carbondata match so we have better efficiency 


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133377209
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java ---
    @@ -0,0 +1,103 @@
    +package org.apache.carbondata.presto.readers;
    --- End diff --
    
    license header is missing


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133370557
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java ---
    @@ -183,7 +183,7 @@ public String getString() throws FilterIllegalMemberException {
                 return parser.format((java.sql.Date) value);
               } else if (value instanceof Long) {
                 if (isLiteral) {
    -              return parser.format(new Timestamp((long) value / 1000));
    +              return parser.format(new Timestamp((long) value));
    --- End diff --
    
    Why need do this change (remove /1000) for getString() ?


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134219553
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataFilterUtil.java ---
    @@ -0,0 +1,233 @@
    +/*
    + * 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
    + */
    +
    +package org.apache.carbondata.presto;
    +
    +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 java.util.stream.Collectors;
    +
    +import org.apache.carbondata.core.metadata.datatype.DataType;
    +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.logical.AndExpression;
    +import org.apache.carbondata.core.scan.expression.logical.OrExpression;
    +
    +import com.facebook.presto.spi.ColumnHandle;
    +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.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 static com.google.common.base.Preconditions.checkArgument;
    +
    +public class CarbondataFilterUtil {
    --- End diff --
    
    done


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134219594
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbonDataTypeConverterImpl.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.carbondata.presto;
    +
    +import java.io.Serializable;
    +import java.nio.charset.Charset;
    +
    +import org.apache.carbondata.core.util.DataTypeConverter;
    +
    +public class CarbonDataTypeConverterImpl implements DataTypeConverter, Serializable {
    --- End diff --
    
    done


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133371582
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java ---
    @@ -183,7 +183,7 @@ public String getString() throws FilterIllegalMemberException {
                 return parser.format((java.sql.Date) value);
               } else if (value instanceof Long) {
                 if (isLiteral) {
    -              return parser.format(new Timestamp((long) value / 1000));
    +              return parser.format(new Timestamp((long) value));
    --- End diff --
    
    @chenliang613 earlier time stamp was getting multiplied by 1000 in carbondata so thats why it was wriiten as 
    -return parser.format(new Timestamp((long) value / 1000));
    but now it is not getting multipled by 1000 so at time of building expression result so now there is no need of dividing timestamp by 1000



---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134464233
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java ---
    @@ -73,27 +60,23 @@ public CarbondataRecordSetProvider(CarbondataConnectorId connectorId, CarbonTabl
     
         CarbondataSplit carbondataSplit =
             checkType(split, CarbondataSplit.class, "split is not class CarbondataSplit");
    -    checkArgument(carbondataSplit.getConnectorId().equals(connectorId), "split is not for this connector");
    +    checkArgument(carbondataSplit.getConnectorId().equals(connectorId),
    +        "split is not for this connector");
     
    -    StringBuffer targetColsBuffer = new StringBuffer();
         String targetCols = "";
         // Convert all columns handles
         ImmutableList.Builder<CarbondataColumnHandle> handles = ImmutableList.builder();
         for (ColumnHandle handle : columns) {
           handles.add(checkType(handle, CarbondataColumnHandle.class, "handle"));
    -      targetColsBuffer.append(((CarbondataColumnHandle) handle).getColumnName()).append(",");
    +      targetCols += ((CarbondataColumnHandle) handle).getColumnName() + ",";
    --- End diff --
    
    @chenliang613 resolved


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

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


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

[GitHub] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134219416
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/processor/AbstractDataBlockIterator.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.carbondata.presto.processor;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.carbondata.common.CarbonIterator;
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.datastore.DataRefNode;
    +import org.apache.carbondata.core.datastore.FileHolder;
    +import org.apache.carbondata.presto.scan.collector.ResultCollectorFactory;
    +import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
    +import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
    +import org.apache.carbondata.core.scan.processor.BlockletIterator;
    +import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
    +import org.apache.carbondata.core.scan.result.AbstractScannedResult;
    +import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
    +import org.apache.carbondata.core.scan.scanner.BlockletScanner;
    +import org.apache.carbondata.core.scan.scanner.impl.FilterScanner;
    +import org.apache.carbondata.core.scan.scanner.impl.NonFilterScanner;
    +import org.apache.carbondata.core.stats.QueryStatisticsModel;
    +
    +/**
    + * This abstract class provides a skeletal implementation of the
    + * Block iterator.
    + */
    +public abstract class AbstractDataBlockIterator extends CarbonIterator<List<Object[]>> {
    --- End diff --
    
    only extending the core abstractdatatblockiterator will not solve our problem,if i extend it then it will pick up result collector from core module that is row based result collector which was initilized in its constructor 


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134219521
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/AbstractAggregatedMemoryContext.java ---
    @@ -0,0 +1,40 @@
    +/*
    + * 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.carbondata.presto.memory;
    +
    +
    +abstract class AbstractAggregatedMemoryContext
    +{
    +  // This class should remain exactly the same as AbstractAggregatedMemoryContext in com.facebook.presto.memory
    +
    +  // AbstractMemoryContext class is only necessary because we need implementations that bridge
    --- End diff --
    
    done


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133377510
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java ---
    @@ -0,0 +1,103 @@
    +package org.apache.carbondata.presto.readers;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +
    +import com.facebook.presto.spi.block.Block;
    +import com.facebook.presto.spi.block.BlockBuilder;
    +import com.facebook.presto.spi.block.BlockBuilderStatus;
    +import com.facebook.presto.spi.type.DecimalType;
    +import com.facebook.presto.spi.type.Decimals;
    +import com.facebook.presto.spi.type.Type;
    +import io.airlift.slice.Slice;
    +
    +import static com.facebook.presto.spi.type.Decimals.encodeUnscaledValue;
    +import static com.facebook.presto.spi.type.Decimals.isShortDecimal;
    +import static com.facebook.presto.spi.type.Decimals.rescale;
    +import static com.google.common.base.Preconditions.checkArgument;
    +import static com.google.common.base.Preconditions.checkState;
    +import static io.airlift.slice.Slices.utf8Slice;
    +import static java.math.RoundingMode.HALF_UP;
    +
    +public class DecimalSliceStreamReader implements StreamReader {
    --- End diff --
    
    This pr is optimizing for column reader, why need to add StreamReader ?  please consider using different PR to implement different features.


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133904356
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/AbstractAggregatedMemoryContext.java ---
    @@ -0,0 +1,40 @@
    +/*
    + * 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.carbondata.presto.memory;
    +
    +
    +abstract class AbstractAggregatedMemoryContext
    +{
    +  // This class should remain exactly the same as AbstractAggregatedMemoryContext in com.facebook.presto.memory
    +
    +  // AbstractMemoryContext class is only necessary because we need implementations that bridge
    --- End diff --
    
    please correct the code size with
    /**
     *
     */


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134460640
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java ---
    @@ -73,27 +60,23 @@ public CarbondataRecordSetProvider(CarbondataConnectorId connectorId, CarbonTabl
     
         CarbondataSplit carbondataSplit =
             checkType(split, CarbondataSplit.class, "split is not class CarbondataSplit");
    -    checkArgument(carbondataSplit.getConnectorId().equals(connectorId), "split is not for this connector");
    +    checkArgument(carbondataSplit.getConnectorId().equals(connectorId),
    +        "split is not for this connector");
     
    -    StringBuffer targetColsBuffer = new StringBuffer();
         String targetCols = "";
         // Convert all columns handles
         ImmutableList.Builder<CarbondataColumnHandle> handles = ImmutableList.builder();
         for (ColumnHandle handle : columns) {
           handles.add(checkType(handle, CarbondataColumnHandle.class, "handle"));
    -      targetColsBuffer.append(((CarbondataColumnHandle) handle).getColumnName()).append(",");
    +      targetCols += ((CarbondataColumnHandle) handle).getColumnName() + ",";
    --- End diff --
    
    Please use StringBuffer to assemble "string"


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134219569
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataFilterUtil.java ---
    @@ -0,0 +1,233 @@
    +/*
    + * 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
    --- End diff --
    
    done


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134219490
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/AbstractAggregatedMemoryContext.java ---
    @@ -0,0 +1,40 @@
    +/*
    + * 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.carbondata.presto.memory;
    +
    +
    +abstract class AbstractAggregatedMemoryContext
    +{
    +  // This class should remain exactly the same as AbstractAggregatedMemoryContext in com.facebook.presto.memory
    +
    +  // AbstractMemoryContext class is only necessary because we need implementations that bridge
    +  // AggregatedMemoryContext with the existing memory tracking APIs in XxxxContext. Once they
    +  // are refactored, there will be only one implementation of this abstract class, and this class
    +  // can be removed.
    +
    +  protected abstract void updateBytes(long bytes);
    +
    +  public AggregatedMemoryContext newAggregatedMemoryContext()
    +  {
    --- End diff --
    
    done


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/217/



---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133901016
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbonDataTypeConverterImpl.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.carbondata.presto;
    +
    +import java.io.Serializable;
    +import java.nio.charset.Charset;
    +
    +import org.apache.carbondata.core.util.DataTypeConverter;
    +
    +public class CarbonDataTypeConverterImpl implements DataTypeConverter, Serializable {
    --- End diff --
    
    I think, don't need to create CarbonDataTypeConverterImpl, you can directly use DataTypeConverterImpl.


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134691842
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/constants/PrestoCommonConstants.java ---
    @@ -0,0 +1,29 @@
    +/*
    + * 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.carbondata.presto.constants;
    +
    +/**
    + * Constants for presto functionality
    + */
    --- End diff --
    
    Please don't add the new class : PrestoCommonConstants.java, keep to define the parameter in CarbondataPageSource.java as below: 
    private static final int ROWS_PER_REQUEST = 4096;


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133375796
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/AggregatedMemoryContext.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed 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
    --- End diff --
    
    The license header is wrong.


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133904744
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/memory/LocalMemoryContext.java ---
    @@ -0,0 +1,43 @@
    +/*
    + * 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.carbondata.presto.memory;
    +
    +import static com.google.common.base.MoreObjects.toStringHelper;
    +import static java.util.Objects.requireNonNull;
    +
    +public class LocalMemoryContext {
    --- End diff --
    
    please add comment for the new class


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133376989
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/readers/BooleanStreamReader.java ---
    @@ -0,0 +1,31 @@
    +package org.apache.carbondata.presto.readers;
    --- End diff --
    
    The license header is missing.


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134692412
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java ---
    @@ -73,27 +60,23 @@ public CarbondataRecordSetProvider(CarbondataConnectorId connectorId, CarbonTabl
     
         CarbondataSplit carbondataSplit =
             checkType(split, CarbondataSplit.class, "split is not class CarbondataSplit");
    -    checkArgument(carbondataSplit.getConnectorId().equals(connectorId), "split is not for this connector");
    +    checkArgument(carbondataSplit.getConnectorId().equals(connectorId),
    +        "split is not for this connector");
     
    -    StringBuffer targetColsBuffer = new StringBuffer();
         String targetCols = "";
         // Convert all columns handles
         ImmutableList.Builder<CarbondataColumnHandle> handles = ImmutableList.builder();
         for (ColumnHandle handle : columns) {
           handles.add(checkType(handle, CarbondataColumnHandle.class, "handle"));
    -      targetColsBuffer.append(((CarbondataColumnHandle) handle).getColumnName()).append(",");
    +      targetCols += ((CarbondataColumnHandle) handle).getColumnName() + ",";
    --- End diff --
    
    i didn't see any code to resolve this 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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133372855
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/constants/CarbonCommonConstants.java ---
    @@ -0,0 +1,1319 @@
    +/*
    --- End diff --
    
    please only keep these CommonConstants which be used by presto module, don't need to copy all.


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133377118
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/readers/BooleanStreamReader.java ---
    @@ -0,0 +1,31 @@
    +package org.apache.carbondata.presto.readers;
    +
    +import java.io.IOException;
    +
    +import com.facebook.presto.spi.block.Block;
    +import com.facebook.presto.spi.block.BlockBuilder;
    +import com.facebook.presto.spi.block.BlockBuilderStatus;
    +import com.facebook.presto.spi.type.Type;
    +
    +public class BooleanStreamReader implements StreamReader {
    --- End diff --
    
    Can you explain why add the class BooleanStreamReader , it is for which purpose?


---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/286/



---
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] carbondata issue #1257: [CARBONDATA-1347] Implemented Columnar Reading Of Da...

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

    https://github.com/apache/carbondata/pull/1257
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/287/



---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r134081827
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/processor/AbstractDataBlockIterator.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.carbondata.presto.processor;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.carbondata.common.CarbonIterator;
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.datastore.DataRefNode;
    +import org.apache.carbondata.core.datastore.FileHolder;
    +import org.apache.carbondata.presto.scan.collector.ResultCollectorFactory;
    +import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
    +import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
    +import org.apache.carbondata.core.scan.processor.BlockletIterator;
    +import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
    +import org.apache.carbondata.core.scan.result.AbstractScannedResult;
    +import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
    +import org.apache.carbondata.core.scan.scanner.BlockletScanner;
    +import org.apache.carbondata.core.scan.scanner.impl.FilterScanner;
    +import org.apache.carbondata.core.scan.scanner.impl.NonFilterScanner;
    +import org.apache.carbondata.core.stats.QueryStatisticsModel;
    +
    +/**
    + * This abstract class provides a skeletal implementation of the
    + * Block iterator.
    + */
    +public abstract class AbstractDataBlockIterator extends CarbonIterator<List<Object[]>> {
    --- End diff --
    
    why need copy this class to presto module, can't directly use :  core/scan/processor/AbstractDataBlockIterator.java ??


---
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] carbondata pull request #1257: [CARBONDATA-1347] Implemented Columnar Readin...

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

    https://github.com/apache/carbondata/pull/1257#discussion_r133901242
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataFilterUtil.java ---
    @@ -0,0 +1,233 @@
    +/*
    + * 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
    --- End diff --
    
    license header is wrong.


---
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.
---