You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/08/07 15:26:31 UTC

[GitHub] [carbondata] ajantha-bhat opened a new pull request #3887: [WIP] Refactor #3773 and support struct type

ajantha-bhat opened a new pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887


   This PR dependent on #3773 
   
    ### Why is this PR needed?
    
    
    ### What changes were proposed in this PR?
   
       
    ### Does this PR introduce any user interface change?
    - No
    - Yes. (please explain the change and update document)
   
    ### Is any new testcase added?
    - No
    - Yes
   
       
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677664277


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677489268


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2066/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476547595



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -282,6 +288,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
           for (int i = 0; i < size; i += DataTypes.INT.getSizeInBytes()) {
             vector.putFloat(rowId++, (max - ByteUtil.toIntLittleEndian(pageData, i)) / floatFactor);
           }
+        } else if (pageDataType == DataTypes.LONG) {

Review comment:
       this means , In some scenario it's storing float data as Long [4bytes to 8 bytes]? Can u pls check the test once




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674011187


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3722/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675571056


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3771/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-682351651


   @kumarvishal09 : PR is ready. Please check and merge once build passed. 
   Also I have prepared the commit message with co-authored by @akkio-97 . Please use the same commit message. Thanks 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674088971


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1988/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r479661505



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -187,6 +197,43 @@ public void fillVector(Object valuesToBeConverted, int size,
             vector.putDecimal(i, value, precision);
           }
         }
+      } else if (pageType == DataTypes.BYTE_ARRAY) {
+        // complex primitive decimal dimension
+        int offset = 0;
+        for (int j = 0; j < size; j++) {
+          // here decimal data will be Length[4 byte], scale[1 byte], value[Length byte]
+          int len = ByteBuffer.wrap(data, offset, DataTypes.INT.getSizeInBytes()).getInt();
+          offset += DataTypes.INT.getSizeInBytes();

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476628989



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
##########
@@ -39,6 +40,15 @@
   public int[] invertedIndex;
   public BitSet deletedRows;
   public DecimalConverterFactory.DecimalConverter decimalConverter;
+  public Stack<CarbonColumnVector> vectorStack = null;

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477308991



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
##########
@@ -376,6 +456,56 @@ private void fillVector(byte[] pageData, CarbonColumnVector vector, DataType vec
           DecimalConverterFactory.DecimalConverter decimalConverter = vectorInfo.decimalConverter;
           decimalConverter.fillVector(pageData, pageSize, vectorInfo, nullBits, pageDataType);
         }
+      } else if (pageDataType == DataTypes.BYTE_ARRAY) {
+        if (vectorDataType == DataTypes.STRING || vectorDataType == DataTypes.BINARY
+            || vectorDataType == DataTypes.VARCHAR) {
+          // for complex primitive string, binary, varchar type
+          int offset = 0;
+          for (int i = 0; i < pageSize; i++) {
+            byte[] stringLen = new byte[DataTypes.INT.getSizeInBytes()];

Review comment:
       this byte array is not required, wrap page data inside byte buffer and updates  the position to get the length of the data
   By this we can avoid creating multiple bytebuffer and byte array . Pls handle all the places 
   
       ```
   ByteBuffer buffer = ByteBuffer.allocate(12);
       buffer.putInt(1);
       buffer.putInt(2);
       buffer.putInt(3);
       buffer.rewind();
       buffer.position(4);
       int anInt = buffer.getInt();
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476450315



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       can we handle this in called method itself, handling in each and class its not correct, pls change this in other classes too




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674336902


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1994/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-673588446


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1975/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r479656097



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -282,6 +288,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
           for (int i = 0; i < size; i += DataTypes.INT.getSizeInBytes()) {
             vector.putFloat(rowId++, (max - ByteUtil.toIntLittleEndian(pageData, i)) / floatFactor);
           }
+        } else if (pageDataType == DataTypes.LONG) {

Review comment:
       Can u pls raise a jira for above issue 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476544468



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       Direct compress needed a different handling because of decimal. So, coundn't handle in decodeDimensionByMeta




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476552867



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -70,10 +73,14 @@
 
   private CarbonColumnVector dictionaryVector;
 
+  private List<CarbonColumnVector> childrenVector;
+
   private LazyPageLoader lazyPage;
 
   private boolean loaded;
 
+  private ArrayList<Integer> childElementsForEachRow = null;

Review comment:
       change this to List<Integer> and null assignment is not required




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674144090


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3732/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476448290



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -282,6 +288,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
           for (int i = 0; i < size; i += DataTypes.INT.getSizeInBytes()) {
             vector.putFloat(rowId++, (max - ByteUtil.toIntLittleEndian(pageData, i)) / floatFactor);
           }
+        } else if (pageDataType == DataTypes.LONG) {

Review comment:
       Why we are handling long here for float data type, for float type page data type cannot be long. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-676478366


   @QiangCai , @jackylk , @kumarvishal09 @ravipesala : please review. 
   Also please go through PR description and open points.
   
   some testcase validation need to improve, now manually verified. will add it by tomorrow. But can start the review.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-682412709


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2162/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-680225124


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3868/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat edited a comment on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat edited a comment on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-678010210


   @marchpure : 
   > Is there a testcase about STRUCT, in which we decode BINARY by BASE64.
   
   Changes are done for presto read (**binary or base 64 decode is needed during the write (converter) step**, not the read step), once decode is done writer will store it as a byte[], presto/spark  just reads the byte[]
   
   > we shall combine 3 commits into single commit. right?
   
   This doesn't matter, once all review done, I can squash or the committer will squash and merge.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] marchpure commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
marchpure commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-678949480


   > @kumarvishal09 @QiangCai please help to review this PR
   
   Why can't I @Kejian-Li?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675662000


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3779/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674756309


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2003/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-683320513


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2174/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-681965099


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3892/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677635399


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2073/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477308991



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
##########
@@ -376,6 +456,56 @@ private void fillVector(byte[] pageData, CarbonColumnVector vector, DataType vec
           DecimalConverterFactory.DecimalConverter decimalConverter = vectorInfo.decimalConverter;
           decimalConverter.fillVector(pageData, pageSize, vectorInfo, nullBits, pageDataType);
         }
+      } else if (pageDataType == DataTypes.BYTE_ARRAY) {
+        if (vectorDataType == DataTypes.STRING || vectorDataType == DataTypes.BINARY
+            || vectorDataType == DataTypes.VARCHAR) {
+          // for complex primitive string, binary, varchar type
+          int offset = 0;
+          for (int i = 0; i < pageSize; i++) {
+            byte[] stringLen = new byte[DataTypes.INT.getSizeInBytes()];

Review comment:
       this byte array is not required, wrap page data inside byte buffer and updates  the position to get the length of the data
   By this we can avoid creating multiple bytebuffer and byte array . Pls handle all the places 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r478227936



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {

Review comment:
       removed this whole class itself and updating vector inside **ColumnarVectorWrapperDirectFactory
             .getDirectVectorWrapperFactory**




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476636246



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -282,6 +288,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
           for (int i = 0; i < size; i += DataTypes.INT.getSizeInBytes()) {
             vector.putFloat(rowId++, (max - ByteUtil.toIntLittleEndian(pageData, i)) / floatFactor);
           }
+        } else if (pageDataType == DataTypes.LONG) {

Review comment:
       I have tested, With current UT itself it is hitting. for [Null, 5.512] it is using long as storage for complex primitive adaptive. Base behavior needs to check. I guess it can be analyzed separately




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-670645884


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1920/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-678010210


   @marchpure : 
   > Is there a testcase about STRUCT, in which we decode BINARY by BASE64.
   Changes are done for presto read (**binary or base 64 decode is needed during the write (converter) step**, not the read step), once decode is done writer will store it as a byte[], presto/spark  just reads the byte[]
   
   > we shall combine 3 commits into single commit. right?
   This doesn't matter, once all review done, I can squash or the committer will squash and merge.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476433166



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {
+      Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+      if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {
+        if (vectorDataType.getName().equals("ARRAY")) {

Review comment:
       can we use DataTypes.isArrayType(vectordatatype)?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] asfgit closed pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677750531


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2075/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677636575


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3814/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r475768161



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {
+    this.childElementsForEachRow = childrenElements;
+  }
+
+  public void setNumberOfChildrenElementsForArray(byte[] childPageData, int pageSize) {

Review comment:
       ```suggestion
     public void setNumberOfChildElementsForArray(byte[] childPageData, int pageSize) {
   ```

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {
+    this.childElementsForEachRow = childrenElements;
+  }
+
+  public void setNumberOfChildrenElementsForArray(byte[] childPageData, int pageSize) {
+    // for complex array type, go through parent page to get the child information
+    ByteBuffer childInfoBuffer = ByteBuffer.wrap(childPageData);
+    ArrayList<Integer> childElementsForEachRow = new ArrayList<>();
+    // osset will be an INT size and value will be another INT size, hence 2 * INT size

Review comment:
       ```suggestion
       // offset will be an INT size and value will be another INT size, hence 2 * INT size
   ```

##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {
+      Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+      if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {
+        if (vectorDataType.getName().equals("ARRAY")) {
+          ArrayList<Integer> childElementsForEachRow =
+              ((CarbonColumnVectorImpl) vector.getColumnVector())
+                  .getNumberOfChildrenElementsInEachRow();
+          int pageSizeNew = 0;

Review comment:
       rename as `newPageSize`

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
##########
@@ -98,6 +98,14 @@ void prepareDimensionAndMeasureColumnVectors() {
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
         allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
+      } else if (queryDimensions[i].getDimension().isComplex()) {

Review comment:
       why this change required? as i can see, its just code moved above in else if

##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();
+    for (StructField child : field.getChildren()) {
+      childrenList.add(new ColumnarVectorWrapperDirect(
+          CarbonVectorBatch.createDirectStreamReader(this.batchSize, child.getDataType(), child)));
+    }
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  Type getType(StructField field) {
+    DataType dataType = field.getDataType();
+    if (dataType == DataTypes.STRING || dataType == DataTypes.VARCHAR) {
+      return VarcharType.VARCHAR;
+    } else if (dataType == DataTypes.SHORT) {
+      return SmallintType.SMALLINT;
+    } else if (dataType == DataTypes.INT) {
+      return IntegerType.INTEGER;
+    } else if (dataType == DataTypes.LONG) {
+      return BigintType.BIGINT;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DoubleType.DOUBLE;
+    } else if (dataType == DataTypes.FLOAT) {
+      return RealType.REAL;
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return BooleanType.BOOLEAN;
+    } else if (dataType == DataTypes.BINARY) {
+      return VarbinaryType.VARBINARY;
+    } else if (dataType == DataTypes.DATE) {
+      return DateType.DATE;
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return TimestampType.TIMESTAMP;
+    } else if (dataType == DataTypes.BYTE) {
+      return TinyintType.TINYINT;
+    } else if (DataTypes.isDecimal(dataType)) {
+      org.apache.carbondata.core.metadata.datatype.DecimalType decimal =
+          (org.apache.carbondata.core.metadata.datatype.DecimalType) dataType;
+      return DecimalType.createDecimalType(decimal.getPrecision(), decimal.getScale());
+    } else if (DataTypes.isArrayType(dataType)) {
+      return new ArrayType(getType(field.getChildren().get(0)));
+    } else if (DataTypes.isStructType(dataType)) {
+      List<RowType.Field> children = new ArrayList<>();
+      for (StructField child : field.getChildren()) {
+        children.add(new RowType.Field(Optional.of(child.getFieldName()), getType(child)));
+      }
+      return RowType.from(children);
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override public Block buildBlock() {
+    return builder.build();
+  }
+
+  @Override public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override public void putObject(int rowId, Object value) {
+    if (value == null) {
+      putNull(rowId);
+    } else {
+      getChildrenVector().get(0).putObject(rowId, value);
+    }
+  }
+
+  public void putComplexObject(List<Integer> offsetVector) {
+    if (type instanceof ArrayType) {
+      Block childBlock = buildChildBlock(getChildrenVector().get(0));
+      // prepare an offset vector with 0 as initial offset
+      int[] offsetVectorArray = new int[offsetVector.size() + 1];
+      for (int i = 1; i <= offsetVector.size(); i++) {
+        offsetVectorArray[i] = offsetVectorArray[i -1] + offsetVector.get(i - 1);
+      }
+      Block arrayBlock = ArrayBlock
+          .fromElementBlock(offsetVector.size(), Optional.ofNullable(null), offsetVectorArray,

Review comment:
       Replace `Optional.ofNullable(null)` with `Optional.empty`

##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {
+      Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+      if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {
+        if (vectorDataType.getName().equals("ARRAY")) {

Review comment:
       can you add `ARRAY` and `STRUCT` as complex string constants?

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {
+    this.childElementsForEachRow = childrenElements;
+  }
+
+  public void setNumberOfChildrenElementsForArray(byte[] childPageData, int pageSize) {
+    // for complex array type, go through parent page to get the child information
+    ByteBuffer childInfoBuffer = ByteBuffer.wrap(childPageData);
+    ArrayList<Integer> childElementsForEachRow = new ArrayList<>();
+    // osset will be an INT size and value will be another INT size, hence 2 * INT size

Review comment:
       if you just mention, the layout of what and how the info is stored in parent data, it will be good for anyone for understanding i feel.

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
##########
@@ -39,6 +40,15 @@
   public int[] invertedIndex;
   public BitSet deletedRows;
   public DecimalConverterFactory.DecimalConverter decimalConverter;
+  public Stack<CarbonColumnVector> vectorStack = null;

Review comment:
       please add comment for this, like you did in the PR description.

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {

Review comment:
       ```suggestion
     public void setNumberOfChildElementsInEachRow(ArrayList<Integer> childrenElements) {
   ```

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {

Review comment:
       ```suggestion
     public ArrayList<Integer> getNumberOfChildElementsInEachRow() {
   ```

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {
+    this.childElementsForEachRow = childrenElements;
+  }
+
+  public void setNumberOfChildrenElementsForArray(byte[] childPageData, int pageSize) {
+    // for complex array type, go through parent page to get the child information
+    ByteBuffer childInfoBuffer = ByteBuffer.wrap(childPageData);
+    ArrayList<Integer> childElementsForEachRow = new ArrayList<>();
+    // osset will be an INT size and value will be another INT size, hence 2 * INT size
+    while (childInfoBuffer.remaining() >= 2 * DataTypes.INT.getSizeInBytes()) {
+      int elements = childInfoBuffer.getInt();
+      if (elements == 0) {
+        break;
+      }
+      childElementsForEachRow.add(elements);
+      // skip offset
+      childInfoBuffer.getInt();
+      if (pageSize == childElementsForEachRow.size()) {
+        break;
+      }
+    }
+    setNumberOfChildrenElementsInEachRow(childElementsForEachRow);
+  }
+
+  public void setNumberOfChildrenElementsForStruct(byte[] childPageData, int pageSize) {

Review comment:
       ```suggestion
     public void setNumberOfChildElementsForStruct(byte[] childPageData, int pageSize) {
   ```

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {
+    this.childElementsForEachRow = childrenElements;
+  }
+
+  public void setNumberOfChildrenElementsForArray(byte[] childPageData, int pageSize) {

Review comment:
       since we are sending the parent page data here from DirectCompressCodec, we can rename `childPageData `as `parentPageData`, as it gives wrong assumption. Am i right? Please change in other method too. 

##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {
+      Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+      if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {
+        if (vectorDataType.getName().equals("ARRAY")) {
+          ArrayList<Integer> childElementsForEachRow =
+              ((CarbonColumnVectorImpl) vector.getColumnVector())
+                  .getNumberOfChildrenElementsInEachRow();
+          int pageSizeNew = 0;
+          for (int val : childElementsForEachRow) {

Review comment:
       can replace val with `noOfChildElements `or `noOfChild`

##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();
+    for (StructField child : field.getChildren()) {
+      childrenList.add(new ColumnarVectorWrapperDirect(
+          CarbonVectorBatch.createDirectStreamReader(this.batchSize, child.getDataType(), child)));
+    }
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  Type getType(StructField field) {
+    DataType dataType = field.getDataType();
+    if (dataType == DataTypes.STRING || dataType == DataTypes.VARCHAR) {
+      return VarcharType.VARCHAR;
+    } else if (dataType == DataTypes.SHORT) {
+      return SmallintType.SMALLINT;
+    } else if (dataType == DataTypes.INT) {
+      return IntegerType.INTEGER;
+    } else if (dataType == DataTypes.LONG) {
+      return BigintType.BIGINT;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DoubleType.DOUBLE;
+    } else if (dataType == DataTypes.FLOAT) {
+      return RealType.REAL;
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return BooleanType.BOOLEAN;
+    } else if (dataType == DataTypes.BINARY) {
+      return VarbinaryType.VARBINARY;
+    } else if (dataType == DataTypes.DATE) {
+      return DateType.DATE;
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return TimestampType.TIMESTAMP;
+    } else if (dataType == DataTypes.BYTE) {
+      return TinyintType.TINYINT;
+    } else if (DataTypes.isDecimal(dataType)) {
+      org.apache.carbondata.core.metadata.datatype.DecimalType decimal =
+          (org.apache.carbondata.core.metadata.datatype.DecimalType) dataType;
+      return DecimalType.createDecimalType(decimal.getPrecision(), decimal.getScale());
+    } else if (DataTypes.isArrayType(dataType)) {
+      return new ArrayType(getType(field.getChildren().get(0)));
+    } else if (DataTypes.isStructType(dataType)) {
+      List<RowType.Field> children = new ArrayList<>();
+      for (StructField child : field.getChildren()) {
+        children.add(new RowType.Field(Optional.of(child.getFieldName()), getType(child)));
+      }
+      return RowType.from(children);
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override public Block buildBlock() {
+    return builder.build();
+  }
+
+  @Override public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override public void putObject(int rowId, Object value) {
+    if (value == null) {
+      putNull(rowId);
+    } else {
+      getChildrenVector().get(0).putObject(rowId, value);
+    }
+  }
+
+  public void putComplexObject(List<Integer> offsetVector) {
+    if (type instanceof ArrayType) {
+      Block childBlock = buildChildBlock(getChildrenVector().get(0));
+      // prepare an offset vector with 0 as initial offset
+      int[] offsetVectorArray = new int[offsetVector.size() + 1];
+      for (int i = 1; i <= offsetVector.size(); i++) {
+        offsetVectorArray[i] = offsetVectorArray[i -1] + offsetVector.get(i - 1);

Review comment:
       correct the formatting  here

##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();
+    for (StructField child : field.getChildren()) {
+      childrenList.add(new ColumnarVectorWrapperDirect(
+          CarbonVectorBatch.createDirectStreamReader(this.batchSize, child.getDataType(), child)));

Review comment:
       `CarbonVectorBatch.createDirectStreamReader` can return null, so to avoid warning, you can use intellij suggestion of Objects.requireNonNull

##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       can we extract line 258 to 265 to a method in `AdaptiveCodec` as these are all same in all Adaptive codec implementations, it will be clean

##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();
+    for (StructField child : field.getChildren()) {
+      childrenList.add(new ColumnarVectorWrapperDirect(
+          CarbonVectorBatch.createDirectStreamReader(this.batchSize, child.getDataType(), child)));
+    }
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  Type getType(StructField field) {
+    DataType dataType = field.getDataType();
+    if (dataType == DataTypes.STRING || dataType == DataTypes.VARCHAR) {
+      return VarcharType.VARCHAR;
+    } else if (dataType == DataTypes.SHORT) {
+      return SmallintType.SMALLINT;
+    } else if (dataType == DataTypes.INT) {
+      return IntegerType.INTEGER;
+    } else if (dataType == DataTypes.LONG) {
+      return BigintType.BIGINT;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DoubleType.DOUBLE;
+    } else if (dataType == DataTypes.FLOAT) {
+      return RealType.REAL;
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return BooleanType.BOOLEAN;
+    } else if (dataType == DataTypes.BINARY) {
+      return VarbinaryType.VARBINARY;
+    } else if (dataType == DataTypes.DATE) {
+      return DateType.DATE;
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return TimestampType.TIMESTAMP;
+    } else if (dataType == DataTypes.BYTE) {
+      return TinyintType.TINYINT;
+    } else if (DataTypes.isDecimal(dataType)) {
+      org.apache.carbondata.core.metadata.datatype.DecimalType decimal =
+          (org.apache.carbondata.core.metadata.datatype.DecimalType) dataType;
+      return DecimalType.createDecimalType(decimal.getPrecision(), decimal.getScale());
+    } else if (DataTypes.isArrayType(dataType)) {
+      return new ArrayType(getType(field.getChildren().get(0)));
+    } else if (DataTypes.isStructType(dataType)) {
+      List<RowType.Field> children = new ArrayList<>();
+      for (StructField child : field.getChildren()) {
+        children.add(new RowType.Field(Optional.of(child.getFieldName()), getType(child)));
+      }
+      return RowType.from(children);
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override public Block buildBlock() {
+    return builder.build();
+  }
+
+  @Override public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override public void putObject(int rowId, Object value) {
+    if (value == null) {
+      putNull(rowId);
+    } else {
+      getChildrenVector().get(0).putObject(rowId, value);

Review comment:
       why always get(0) for put object? shouldn't be based on the datatype of struct and array as this is common for complexStreamReaders?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674036894


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1985/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675778572


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3780/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674143581


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1993/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r475389715



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {

Review comment:
       how about to create a new java file 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477058306



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -282,6 +288,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
           for (int i = 0; i < size; i += DataTypes.INT.getSizeInBytes()) {
             vector.putFloat(rowId++, (max - ByteUtil.toIntLittleEndian(pageData, i)) / floatFactor);
           }
+        } else if (pageDataType == DataTypes.LONG) {

Review comment:
       For this, I have checked 
   If No complex type, (if it is just primitive type) same values goes to DirectCompress, not adaptive. But for complex primitive it goes to adaptive because of below code. And as min max is stored as double precision. Long is chosen for this.
   
   
   `DefaultEncodingFactory#selectCodecByAlgorithmForFloating()`
   
   ```
   } else if (decimalCount < 0 && !isComplexPrimitive) {
         return new DirectCompressCodec(DataTypes.DOUBLE);
       } else {
         return getColumnPageCodec(stats, isComplexPrimitive, columnSpec, srcDataType, maxValue,
             minValue, decimalCount, absMaxValue);
       }
   ```
   
   I don't know (remember) why complex primitive should not enter direct compress.  why that check is explicitly added.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-683368433


   LGTM


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476440493



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       yeah, correct, and no need to try from my side..!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477289221



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,65 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  public static VectorUtil checkAndUpdateToChildVector(ColumnVectorInfo vectorInfo, int pageSize,
+      CarbonColumnVector vector, DataType vectorDataType) {
+    VectorUtil vectorUtil = new VectorUtil(pageSize, vector, vectorDataType);
+    Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+    // check and update to child vector info
+    if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {

Review comment:
       use Objects.nonNull for not null case and for null use Objects.isnull, pls handle in all applicable places 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674337019


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3733/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675783198


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2038/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675366072


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3761/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-676189511


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2048/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat edited a comment on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat edited a comment on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-678010210






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r478855152



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {
+    this.childElementsForEachRow = childrenElements;
+  }
+
+  public void setNumberOfChildrenElementsForArray(byte[] childPageData, int pageSize) {
+    // for complex array type, go through parent page to get the child information
+    ByteBuffer childInfoBuffer = ByteBuffer.wrap(childPageData);
+    ArrayList<Integer> childElementsForEachRow = new ArrayList<>();
+    // osset will be an INT size and value will be another INT size, hence 2 * INT size

Review comment:
       added




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-676563702


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2052/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674103002


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3728/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-682393514


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3903/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r479656884



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -187,6 +197,43 @@ public void fillVector(Object valuesToBeConverted, int size,
             vector.putDecimal(i, value, precision);
           }
         }
+      } else if (pageType == DataTypes.BYTE_ARRAY) {
+        // complex primitive decimal dimension
+        int offset = 0;
+        for (int j = 0; j < size; j++) {
+          // here decimal data will be Length[4 byte], scale[1 byte], value[Length byte]
+          int len = ByteBuffer.wrap(data, offset, DataTypes.INT.getSizeInBytes()).getInt();
+          offset += DataTypes.INT.getSizeInBytes();

Review comment:
       DataTypes.INT.getSizeInBytes), It’s better to assign to some variable and use it inside loop, instead calling method every time 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476433166



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {
+      Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+      if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {
+        if (vectorDataType.getName().equals("ARRAY")) {

Review comment:
       can we can DataTypes.isArrayType(vectordatatype)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-683321068


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3915/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476450315



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       can we handle this in caller method itself, handling in each and every class its not correct, pls change this in other classes too




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] marchpure commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
marchpure commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-678949876


   > @kumarvishal09 @QiangCai please help to review this PR
   
   Add @Kejian-Li to review this PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675365569


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2021/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-681894212


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3889/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674134933


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1992/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476450315



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       can we handle this in caller method itself, handling in each and class its not correct, pls change this in other classes too




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat edited a comment on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat edited a comment on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-678010210


   @marchpure : 
   > Is there a testcase about STRUCT, in which we decode BINARY by BASE64.
   
   Changes are done for presto read (**hex or base64 decode for binary is needed during the write (converter) step**, not at the read step), once decode is done writer will store it as a byte[], presto/spark  just reads the byte[]
   
   > we shall combine 3 commits into single commit. right?
   
   This doesn't matter, once all review done, I can squash or the committer will squash and merge.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r478374714



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,65 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  public static VectorUtil checkAndUpdateToChildVector(ColumnVectorInfo vectorInfo, int pageSize,
+      CarbonColumnVector vector, DataType vectorDataType) {
+    VectorUtil vectorUtil = new VectorUtil(pageSize, vector, vectorDataType);
+    Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+    // check and update to child vector info
+    if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {

Review comment:
       Objects.isnull and Objects.nonNull is internally just doing the same. 
   For just "if check" it is not much useful (We have not used in our code for if checks also anywhere) 
   **I will use it in the future if I use streams.**
   
   ![Screenshot from 2020-08-27 17-43-51](https://user-images.githubusercontent.com/5889404/91441253-7d649d00-e88d-11ea-8d30-aa73af1cd7d6.png)
   
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-681914750


   > So for complex type user must configure table page size to get the better query performance while creating the table.
   
   @kumarvishal09: I too strongly agree that if user using complex type with huge data in one row, page size is mandatory to use. 
   If I make it default now, some testcase validation will fail (example CLI test case which are checking page size), I will handle in another PR. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r478856181



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,65 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  public static VectorUtil checkAndUpdateToChildVector(ColumnVectorInfo vectorInfo, int pageSize,
+      CarbonColumnVector vector, DataType vectorDataType) {
+    VectorUtil vectorUtil = new VectorUtil(pageSize, vector, vectorDataType);
+    Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+    // check and update to child vector info
+    if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {

Review comment:
       **Also initialized the stack always, checking for empty now. no null check is present**




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675713570


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2037/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-676550943


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3795/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674026354


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1984/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r479657086



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();

Review comment:
       Better use Interface type, List instead of ArrayList




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675382970


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3762/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476625159



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnVector.java
##########
@@ -114,4 +115,20 @@
 
   void setLazyPage(LazyPageLoader lazyPage);
 
+  // Add default implementation for interface,
+  // to avoid implementing presto required functions for spark or core module.
+  default List<CarbonColumnVector> getChildrenVector() {
+    return null;

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -70,10 +73,14 @@
 
   private CarbonColumnVector dictionaryVector;
 
+  private List<CarbonColumnVector> childrenVector;
+
   private LazyPageLoader lazyPage;
 
   private boolean loaded;
 
+  private ArrayList<Integer> childElementsForEachRow = null;

Review comment:
       yeah. done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r479656578



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -282,6 +288,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
           for (int i = 0; i < size; i += DataTypes.INT.getSizeInBytes()) {
             vector.putFloat(rowId++, (max - ByteUtil.toIntLittleEndian(pageData, i)) / floatFactor);
           }
+        } else if (pageDataType == DataTypes.LONG) {

Review comment:
       jira is created 
   
   https://issues.apache.org/jira/browse/CARBONDATA-3965




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-681891693


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2148/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476544468



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       Direct compress needed a different handling because of decimal. So, had to update in each




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476439065



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {

Review comment:
       change this to static and pass all the fields required to build this class as a params and create instance of this class here in this method, constructor make it private 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477040770



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       Also, decodeAndFillVector takes vectorInfo, not the vectors. So, all interface need to change if I do this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674102474


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1989/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675382272


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2022/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674764749


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3743/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r478367442



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
##########
@@ -376,6 +456,56 @@ private void fillVector(byte[] pageData, CarbonColumnVector vector, DataType vec
           DecimalConverterFactory.DecimalConverter decimalConverter = vectorInfo.decimalConverter;
           decimalConverter.fillVector(pageData, pageSize, vectorInfo, nullBits, pageDataType);
         }
+      } else if (pageDataType == DataTypes.BYTE_ARRAY) {
+        if (vectorDataType == DataTypes.STRING || vectorDataType == DataTypes.BINARY
+            || vectorDataType == DataTypes.VARCHAR) {
+          // for complex primitive string, binary, varchar type
+          int offset = 0;
+          for (int i = 0; i < pageSize; i++) {
+            byte[] stringLen = new byte[DataTypes.INT.getSizeInBytes()];

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
##########
@@ -98,6 +98,9 @@
    */
   protected CarbonIterator queryIterator;
 
+  // Size of the ReusableDataBuffer based on the number of dimension projection columns
+  protected int reusableDimensionBufferSize = 0;

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
##########
@@ -153,6 +154,9 @@
 
   private ReusableDataBuffer[] measureReusableBuffer;
 
+  // index used by dimensionReusableBuffer
+  int dimensionReusableBufferIndex = 0;

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476428856



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       This is java, so here the logic is to assign multiple variables, hence it is already optimized to fill those info in util class. here is just assignment. 
   
   You can try and let me know !




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674035748


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3724/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] marchpure removed a comment on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
marchpure removed a comment on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-678949480


   > @kumarvishal09 @QiangCai please help to review this PR
   
   Why can't I @Kejian-Li?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674012650


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1983/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-670646477


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3659/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] brijoobopanna commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
brijoobopanna commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-678945401


   @kumarvishal09 @QiangCai please help to review this PR


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476423462



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {

Review comment:
       This is required only  for adaptive classes. Hence better to keep as sublass I feel.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476625494



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();
+    for (StructField child : field.getChildren()) {
+      childrenList.add(new ColumnarVectorWrapperDirect(
+          CarbonVectorBatch.createDirectStreamReader(this.batchSize, child.getDataType(), child)));
+    }
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  Type getType(StructField field) {
+    DataType dataType = field.getDataType();
+    if (dataType == DataTypes.STRING || dataType == DataTypes.VARCHAR) {
+      return VarcharType.VARCHAR;
+    } else if (dataType == DataTypes.SHORT) {
+      return SmallintType.SMALLINT;
+    } else if (dataType == DataTypes.INT) {
+      return IntegerType.INTEGER;
+    } else if (dataType == DataTypes.LONG) {
+      return BigintType.BIGINT;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DoubleType.DOUBLE;
+    } else if (dataType == DataTypes.FLOAT) {
+      return RealType.REAL;
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return BooleanType.BOOLEAN;
+    } else if (dataType == DataTypes.BINARY) {
+      return VarbinaryType.VARBINARY;
+    } else if (dataType == DataTypes.DATE) {
+      return DateType.DATE;
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return TimestampType.TIMESTAMP;
+    } else if (dataType == DataTypes.BYTE) {
+      return TinyintType.TINYINT;
+    } else if (DataTypes.isDecimal(dataType)) {
+      org.apache.carbondata.core.metadata.datatype.DecimalType decimal =
+          (org.apache.carbondata.core.metadata.datatype.DecimalType) dataType;
+      return DecimalType.createDecimalType(decimal.getPrecision(), decimal.getScale());
+    } else if (DataTypes.isArrayType(dataType)) {
+      return new ArrayType(getType(field.getChildren().get(0)));
+    } else if (DataTypes.isStructType(dataType)) {
+      List<RowType.Field> children = new ArrayList<>();
+      for (StructField child : field.getChildren()) {
+        children.add(new RowType.Field(Optional.of(child.getFieldName()), getType(child)));
+      }
+      return RowType.from(children);
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override public Block buildBlock() {
+    return builder.build();
+  }
+
+  @Override public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override public void putObject(int rowId, Object value) {
+    if (value == null) {
+      putNull(rowId);
+    } else {
+      getChildrenVector().get(0).putObject(rowId, value);
+    }
+  }
+
+  public void putComplexObject(List<Integer> offsetVector) {
+    if (type instanceof ArrayType) {
+      Block childBlock = buildChildBlock(getChildrenVector().get(0));
+      // prepare an offset vector with 0 as initial offset
+      int[] offsetVectorArray = new int[offsetVector.size() + 1];
+      for (int i = 1; i <= offsetVector.size(); i++) {
+        offsetVectorArray[i] = offsetVectorArray[i -1] + offsetVector.get(i - 1);

Review comment:
       wonder why checkstyle is not catching 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677744032


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3816/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] marchpure edited a comment on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
marchpure edited a comment on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677786847


   Is there a testcase about STRUCT<BINARY>, in which we decode BINARY by BASE64.
   we shall combine 3 commits into single commit. right?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476550903



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnVector.java
##########
@@ -114,4 +115,20 @@
 
   void setLazyPage(LazyPageLoader lazyPage);
 
+  // Add default implementation for interface,
+  // to avoid implementing presto required functions for spark or core module.
+  default List<CarbonColumnVector> getChildrenVector() {
+    return null;

Review comment:
       return empty list from here 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476429529



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {
+    this.childElementsForEachRow = childrenElements;
+  }
+
+  public void setNumberOfChildrenElementsForArray(byte[] childPageData, int pageSize) {
+    // for complex array type, go through parent page to get the child information
+    ByteBuffer childInfoBuffer = ByteBuffer.wrap(childPageData);
+    ArrayList<Integer> childElementsForEachRow = new ArrayList<>();
+    // osset will be an INT size and value will be another INT size, hence 2 * INT size

Review comment:
       Need to have write flow understanding for this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] marchpure commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
marchpure commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677786847


   Is there a testcase about STRUCT, in which we decode BINARY by BASE64.
   we shall combine 3 commits into single commit. right?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476426320



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
##########
@@ -98,6 +98,14 @@ void prepareDimensionAndMeasureColumnVectors() {
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
         allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
+      } else if (queryDimensions[i].getDimension().isComplex()) {

Review comment:
       If you see carefully, line 109 is queryDimensions[i].getDimension().getDataType() != DataTypes.DATE, complex is also !=Date. so it was never entering the expected branch




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-681845433


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2145/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477289221



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,65 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  public static VectorUtil checkAndUpdateToChildVector(ColumnVectorInfo vectorInfo, int pageSize,
+      CarbonColumnVector vector, DataType vectorDataType) {
+    VectorUtil vectorUtil = new VectorUtil(pageSize, vector, vectorDataType);
+    Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+    // check and update to child vector info
+    if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {

Review comment:
       use Objects.nonNull for not null case and for null use Objects.isnull




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477313220



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
##########
@@ -98,6 +98,9 @@
    */
   protected CarbonIterator queryIterator;
 
+  // Size of the ReusableDataBuffer based on the number of dimension projection columns
+  protected int reusableDimensionBufferSize = 0;

Review comment:
       by default Int value is 0, remove it 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477292131



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,65 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  public static VectorUtil checkAndUpdateToChildVector(ColumnVectorInfo vectorInfo, int pageSize,
+      CarbonColumnVector vector, DataType vectorDataType) {
+    VectorUtil vectorUtil = new VectorUtil(pageSize, vector, vectorDataType);
+    Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+    // check and update to child vector info
+    if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {
+      if (DataTypes.isArrayType(vectorDataType)) {
+        List<Integer> childElementsCountForEachRow =
+            ((CarbonColumnVectorImpl) vector.getColumnVector())
+                .getNumberOfChildrenElementsInEachRow();
+        int newPageSize = 0;
+        for (int childElementsCount : childElementsCountForEachRow) {
+          newPageSize += childElementsCount;
+        }
+        vectorUtil.setPageSize(newPageSize);
+      }
+      // child vector flow, so fill the child vector
+      CarbonColumnVector childVector = vectorStack.pop();
+      vectorUtil.setVector(childVector);
+      vectorUtil.setVectorDataType(childVector.getType());
+    }
+    return vectorUtil;
+  }
+
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    private VectorUtil(int pageSize, CarbonColumnVector vector, DataType vectorDataType) {
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {

Review comment:
       we can use org.apache.commons.lang3.tuple class for returning 3 args from one method , pls check if you can use here 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674026676


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3723/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-681969792


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2151/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476625603



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();
+    for (StructField child : field.getChildren()) {
+      childrenList.add(new ColumnarVectorWrapperDirect(
+          CarbonVectorBatch.createDirectStreamReader(this.batchSize, child.getDataType(), child)));
+    }
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  Type getType(StructField field) {
+    DataType dataType = field.getDataType();
+    if (dataType == DataTypes.STRING || dataType == DataTypes.VARCHAR) {
+      return VarcharType.VARCHAR;
+    } else if (dataType == DataTypes.SHORT) {
+      return SmallintType.SMALLINT;
+    } else if (dataType == DataTypes.INT) {
+      return IntegerType.INTEGER;
+    } else if (dataType == DataTypes.LONG) {
+      return BigintType.BIGINT;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DoubleType.DOUBLE;
+    } else if (dataType == DataTypes.FLOAT) {
+      return RealType.REAL;
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return BooleanType.BOOLEAN;
+    } else if (dataType == DataTypes.BINARY) {
+      return VarbinaryType.VARBINARY;
+    } else if (dataType == DataTypes.DATE) {
+      return DateType.DATE;
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return TimestampType.TIMESTAMP;
+    } else if (dataType == DataTypes.BYTE) {
+      return TinyintType.TINYINT;
+    } else if (DataTypes.isDecimal(dataType)) {
+      org.apache.carbondata.core.metadata.datatype.DecimalType decimal =
+          (org.apache.carbondata.core.metadata.datatype.DecimalType) dataType;
+      return DecimalType.createDecimalType(decimal.getPrecision(), decimal.getScale());
+    } else if (DataTypes.isArrayType(dataType)) {
+      return new ArrayType(getType(field.getChildren().get(0)));
+    } else if (DataTypes.isStructType(dataType)) {
+      List<RowType.Field> children = new ArrayList<>();
+      for (StructField child : field.getChildren()) {
+        children.add(new RowType.Field(Optional.of(child.getFieldName()), getType(child)));
+      }
+      return RowType.from(children);
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override public Block buildBlock() {
+    return builder.build();
+  }
+
+  @Override public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override public void putObject(int rowId, Object value) {
+    if (value == null) {
+      putNull(rowId);
+    } else {
+      getChildrenVector().get(0).putObject(rowId, value);

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-677495917


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3807/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476427767



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();
+    for (StructField child : field.getChildren()) {
+      childrenList.add(new ColumnarVectorWrapperDirect(
+          CarbonVectorBatch.createDirectStreamReader(this.batchSize, child.getDataType(), child)));
+    }
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  Type getType(StructField field) {
+    DataType dataType = field.getDataType();
+    if (dataType == DataTypes.STRING || dataType == DataTypes.VARCHAR) {
+      return VarcharType.VARCHAR;
+    } else if (dataType == DataTypes.SHORT) {
+      return SmallintType.SMALLINT;
+    } else if (dataType == DataTypes.INT) {
+      return IntegerType.INTEGER;
+    } else if (dataType == DataTypes.LONG) {
+      return BigintType.BIGINT;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DoubleType.DOUBLE;
+    } else if (dataType == DataTypes.FLOAT) {
+      return RealType.REAL;
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return BooleanType.BOOLEAN;
+    } else if (dataType == DataTypes.BINARY) {
+      return VarbinaryType.VARBINARY;
+    } else if (dataType == DataTypes.DATE) {
+      return DateType.DATE;
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return TimestampType.TIMESTAMP;
+    } else if (dataType == DataTypes.BYTE) {
+      return TinyintType.TINYINT;
+    } else if (DataTypes.isDecimal(dataType)) {
+      org.apache.carbondata.core.metadata.datatype.DecimalType decimal =
+          (org.apache.carbondata.core.metadata.datatype.DecimalType) dataType;
+      return DecimalType.createDecimalType(decimal.getPrecision(), decimal.getScale());
+    } else if (DataTypes.isArrayType(dataType)) {
+      return new ArrayType(getType(field.getChildren().get(0)));
+    } else if (DataTypes.isStructType(dataType)) {
+      List<RowType.Field> children = new ArrayList<>();
+      for (StructField child : field.getChildren()) {
+        children.add(new RowType.Field(Optional.of(child.getFieldName()), getType(child)));
+      }
+      return RowType.from(children);
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override public Block buildBlock() {
+    return builder.build();
+  }
+
+  @Override public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override public void putObject(int rowId, Object value) {
+    if (value == null) {
+      putNull(rowId);
+    } else {
+      getChildrenVector().get(0).putObject(rowId, value);

Review comment:
       This is can be unsupported error, as we use only putComplexObject




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476624952



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {
+      Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+      if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {
+        if (vectorDataType.getName().equals("ARRAY")) {

Review comment:
       yes. I can modify

##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private ColumnVectorInfo vectorInfo;
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    public VectorUtil(ColumnVectorInfo vectorInfo, int pageSize, CarbonColumnVector vector,
+        DataType vectorDataType) {
+      this.vectorInfo = vectorInfo;
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {
+      return pageSize;
+    }
+
+    public CarbonColumnVector getVector() {
+      return vector;
+    }
+
+    public DataType getVectorDataType() {
+      return vectorDataType;
+    }
+
+    public VectorUtil checkAndUpdateToChildVector() {

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-681843843


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3886/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674087152


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3727/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r478855676



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -255,6 +255,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       CarbonColumnVector vector = vectorInfo.vector;
       BitSet deletedRows = vectorInfo.deletedRows;
       DataType vectorDataType = vector.getType();
+      VectorUtil vectorUtil = new VectorUtil(vectorInfo, pageSize, vector, vectorDataType)

Review comment:
       removed this whole class itself and updating vector inside ColumnarVectorWrapperDirectFactory
   .getDirectVectorWrapperFactory




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r478227589



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,65 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  public static VectorUtil checkAndUpdateToChildVector(ColumnVectorInfo vectorInfo, int pageSize,
+      CarbonColumnVector vector, DataType vectorDataType) {
+    VectorUtil vectorUtil = new VectorUtil(pageSize, vector, vectorDataType);
+    Stack<CarbonColumnVector> vectorStack = vectorInfo.getVectorStack();
+    // check and update to child vector info
+    if (vectorStack != null && vectorStack.peek() != null && vectorDataType.isComplexType()) {
+      if (DataTypes.isArrayType(vectorDataType)) {
+        List<Integer> childElementsCountForEachRow =
+            ((CarbonColumnVectorImpl) vector.getColumnVector())
+                .getNumberOfChildrenElementsInEachRow();
+        int newPageSize = 0;
+        for (int childElementsCount : childElementsCountForEachRow) {
+          newPageSize += childElementsCount;
+        }
+        vectorUtil.setPageSize(newPageSize);
+      }
+      // child vector flow, so fill the child vector
+      CarbonColumnVector childVector = vectorStack.pop();
+      vectorUtil.setVector(childVector);
+      vectorUtil.setVectorDataType(childVector.getType());
+    }
+    return vectorUtil;
+  }
+
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {
+    private int pageSize;
+    private CarbonColumnVector vector;
+    private DataType vectorDataType;
+
+    private VectorUtil(int pageSize, CarbonColumnVector vector, DataType vectorDataType) {
+      this.pageSize = pageSize;
+      this.vector = vector;
+      this.vectorDataType = vectorDataType;
+    }
+
+    public int getPageSize() {

Review comment:
       removed this whole class itself and updating vector inside **ColumnarVectorWrapperDirectFactory
             .getDirectVectorWrapperFactory**

##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
##########
@@ -260,4 +265,52 @@ protected String debugInfo() {
     return this.toString();
   }
 
+  // Utility class to update current vector to child vector in case of complex type handling
+  public static class VectorUtil {

Review comment:
       removed this whole class itself and updating vector inside **ColumnarVectorWrapperDirectFactory
             .getDirectVectorWrapperFactory**




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-680904506


   @ajantha-bhat By default Row filter push down is false, In case of complex type for eg: array of string number of records can be much more than 32k, so filling in one shot can be a problem because creating a big array will be an overhead. 
   
   + In case of direct fill we have ResuableDataBuffer to create one big byte array/column for one blocklet processing  and reuse for all the page for a column inside, this wont be useful because number of element in case of complex type can vary, and byte array size will change every page and even creating a big byte array for one page when number of element in child is more can degrade query performance. 
   
   So for complex type user must configure table page size to get the better query performance while creating the table.
   
   If number of row is page is higher we can  avoid ResuableDataBuffer it may be become overhead. 
   
   Some comments:
   1. Pls refactor the code, avoid duplicate code if you can 
   2. Pls add comments.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674135324


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3731/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r479661551



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();

Review comment:
       check and handled all the newly added code




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-676183932


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3790/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477308991



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
##########
@@ -376,6 +456,56 @@ private void fillVector(byte[] pageData, CarbonColumnVector vector, DataType vec
           DecimalConverterFactory.DecimalConverter decimalConverter = vectorInfo.decimalConverter;
           decimalConverter.fillVector(pageData, pageSize, vectorInfo, nullBits, pageDataType);
         }
+      } else if (pageDataType == DataTypes.BYTE_ARRAY) {
+        if (vectorDataType == DataTypes.STRING || vectorDataType == DataTypes.BINARY
+            || vectorDataType == DataTypes.VARCHAR) {
+          // for complex primitive string, binary, varchar type
+          int offset = 0;
+          for (int i = 0; i < pageSize; i++) {
+            byte[] stringLen = new byte[DataTypes.INT.getSizeInBytes()];

Review comment:
       this byte array is not required, wrap page data inside byte buffer and updates  the position to get the length of the data
   By this we can avoid creating multiple bytebuffer and byte array 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-673588956


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3714/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476438157



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ComplexTypeStreamReader.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import io.prestosql.spi.block.ArrayBlock;
+import io.prestosql.spi.block.RowBlock;
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+import org.apache.carbondata.presto.ColumnarVectorWrapperDirect;
+
+/**
+ * Class to read the complex type Stream [array/struct/map]
+ */
+
+public class ComplexTypeStreamReader extends CarbonColumnVectorImpl
+    implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+
+  public ComplexTypeStreamReader(int batchSize, StructField field) {
+    super(batchSize, field.getDataType());
+    this.batchSize = batchSize;
+    this.type = getType(field);
+    ArrayList<CarbonColumnVector> childrenList = new ArrayList<>();
+    for (StructField child : field.getChildren()) {
+      childrenList.add(new ColumnarVectorWrapperDirect(
+          CarbonVectorBatch.createDirectStreamReader(this.batchSize, child.getDataType(), child)));
+    }
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  Type getType(StructField field) {
+    DataType dataType = field.getDataType();
+    if (dataType == DataTypes.STRING || dataType == DataTypes.VARCHAR) {
+      return VarcharType.VARCHAR;
+    } else if (dataType == DataTypes.SHORT) {
+      return SmallintType.SMALLINT;
+    } else if (dataType == DataTypes.INT) {
+      return IntegerType.INTEGER;
+    } else if (dataType == DataTypes.LONG) {
+      return BigintType.BIGINT;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DoubleType.DOUBLE;
+    } else if (dataType == DataTypes.FLOAT) {
+      return RealType.REAL;
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return BooleanType.BOOLEAN;
+    } else if (dataType == DataTypes.BINARY) {
+      return VarbinaryType.VARBINARY;
+    } else if (dataType == DataTypes.DATE) {
+      return DateType.DATE;
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return TimestampType.TIMESTAMP;
+    } else if (dataType == DataTypes.BYTE) {
+      return TinyintType.TINYINT;
+    } else if (DataTypes.isDecimal(dataType)) {
+      org.apache.carbondata.core.metadata.datatype.DecimalType decimal =
+          (org.apache.carbondata.core.metadata.datatype.DecimalType) dataType;
+      return DecimalType.createDecimalType(decimal.getPrecision(), decimal.getScale());
+    } else if (DataTypes.isArrayType(dataType)) {
+      return new ArrayType(getType(field.getChildren().get(0)));
+    } else if (DataTypes.isStructType(dataType)) {
+      List<RowType.Field> children = new ArrayList<>();
+      for (StructField child : field.getChildren()) {
+        children.add(new RowType.Field(Optional.of(child.getFieldName()), getType(child)));
+      }
+      return RowType.from(children);
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override public Block buildBlock() {
+    return builder.build();
+  }
+
+  @Override public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override public void putObject(int rowId, Object value) {
+    if (value == null) {
+      putNull(rowId);
+    } else {
+      getChildrenVector().get(0).putObject(rowId, value);

Review comment:
       ok, if we use `putComplexObject` , then better no need of any implementation for this method, as you said can be changed to unsupported error.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674898812


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3747/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-674907332


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2006/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-680226920


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2127/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3887: [WIP] Refactor #3773 and support struct type

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#issuecomment-675569959


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2029/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477313518



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
##########
@@ -153,6 +154,9 @@
 
   private ReusableDataBuffer[] measureReusableBuffer;
 
+  // index used by dimensionReusableBuffer
+  int dimensionReusableBufferIndex = 0;

Review comment:
       private int dimensionReusableBufferIndex = 0;




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r479661536



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -187,6 +197,43 @@ public void fillVector(Object valuesToBeConverted, int size,
             vector.putDecimal(i, value, precision);
           }
         }
+      } else if (pageType == DataTypes.BYTE_ARRAY) {
+        // complex primitive decimal dimension
+        int offset = 0;
+        for (int j = 0; j < size; j++) {
+          // here decimal data will be Length[4 byte], scale[1 byte], value[Length byte]
+          int len = ByteBuffer.wrap(data, offset, DataTypes.INT.getSizeInBytes()).getInt();
+          offset += DataTypes.INT.getSizeInBytes();

Review comment:
       handled all the 5 files including base code




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476439569



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonColumnVectorImpl.java
##########
@@ -102,6 +109,57 @@ public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
 
   }
 
+  @Override
+  public List<CarbonColumnVector> getChildrenVector() {
+    return childrenVector;
+  }
+
+  public void setChildrenVector(ArrayList<CarbonColumnVector> childrenVector) {
+    this.childrenVector = childrenVector;
+  }
+
+  public ArrayList<Integer> getNumberOfChildrenElementsInEachRow() {
+    return childElementsForEachRow;
+  }
+
+  public void setNumberOfChildrenElementsInEachRow(ArrayList<Integer> childrenElements) {
+    this.childElementsForEachRow = childrenElements;
+  }
+
+  public void setNumberOfChildrenElementsForArray(byte[] childPageData, int pageSize) {
+    // for complex array type, go through parent page to get the child information
+    ByteBuffer childInfoBuffer = ByteBuffer.wrap(childPageData);
+    ArrayList<Integer> childElementsForEachRow = new ArrayList<>();
+    // osset will be an INT size and value will be another INT size, hence 2 * INT size

Review comment:
       yeah, i agree, but its for developer purpose, whoever has write flow understanding,  it will be easy for them to understand, as we have many such places with very good comments written by @kumarvishal09 and others. I suggest to add here.!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kumarvishal09 commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
kumarvishal09 commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r477313518



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
##########
@@ -153,6 +154,9 @@
 
   private ReusableDataBuffer[] measureReusableBuffer;
 
+  // index used by dimensionReusableBuffer
+  int dimensionReusableBufferIndex = 0;

Review comment:
       change private int dimensionReusableBufferIndex;




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3887: [CARBONDATA-3830] Support Array and Struct of all primitive type reading from presto

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3887:
URL: https://github.com/apache/carbondata/pull/3887#discussion_r476450601



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
##########
@@ -282,6 +288,12 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
           for (int i = 0; i < size; i += DataTypes.INT.getSizeInBytes()) {
             vector.putFloat(rowId++, (max - ByteUtil.toIntLittleEndian(pageData, i)) / floatFactor);
           }
+        } else if (pageDataType == DataTypes.LONG) {

Review comment:
       For AdaptiveFloating codec, float data was coming as long for complex primitive.
   So added for delta flow also.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org