[GitHub] [carbondata] akkio-97 opened a new pull request #3773: [CARBONDATA-3830]Presto complex columns read support

classic Classic list List threaded Threaded
112 messages Options
123456
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akkio-97 commented on a change in pull request #3773: [CARBONDATA-3830]Presto complex columns read support

GitBox

akkio-97 commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r462856119



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ArrayStreamReader.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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 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.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+
+/**
+ * Class to read the Array Stream
+ */
+
+public class ArrayStreamReader extends CarbonColumnVectorImpl implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+  Block childBlock = null;
+  private int index = 0;
+
+  public ArrayStreamReader(int batchSize, DataType dataType, StructField field) {
+    super(batchSize, dataType);
+    this.batchSize = batchSize;
+    this.type = getArrayOfType(field, dataType);
+    setChildrenVector(
+        CarbonVectorBatch.createDirectStreamReader(this.batchSize, field.getDataType(), field));
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public void setIndex(int index) {
+    this.index = index;
+  }
+
+  public String getDataTypeName() {
+    return "ARRAY";
+  }
+
+  Type getArrayOfType(StructField field, DataType dataType) {
+    if (dataType == DataTypes.STRING) {
+      return new ArrayType(VarcharType.VARCHAR);
+    } else if (dataType == DataTypes.INT) {
+      return new ArrayType(IntegerType.INTEGER);
+    } else if (dataType == DataTypes.LONG) {
+      return new ArrayType(BigintType.BIGINT);
+    } else if (dataType == DataTypes.DOUBLE) {
+      return new ArrayType(DoubleType.DOUBLE);
+    } else if (dataType == DataTypes.FLOAT) {
+      return new ArrayType(RealType.REAL);
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return new ArrayType(BooleanType.BOOLEAN);
+    } else {
+      StructField childField = field.getChildren().get(0);
+      return new ArrayType(getArrayOfType(childField, childField.getDataType()));

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akkio-97 commented on a change in pull request #3773: [CARBONDATA-3830]Presto complex columns read support

GitBox
In reply to this post by GitBox

akkio-97 commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r462856584



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ArrayStreamReader.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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 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.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+
+/**
+ * Class to read the Array Stream
+ */
+
+public class ArrayStreamReader extends CarbonColumnVectorImpl implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+  Block childBlock = null;
+  private int index = 0;
+
+  public ArrayStreamReader(int batchSize, DataType dataType, StructField field) {
+    super(batchSize, dataType);
+    this.batchSize = batchSize;
+    this.type = getArrayOfType(field, dataType);
+    setChildrenVector(
+        CarbonVectorBatch.createDirectStreamReader(this.batchSize, field.getDataType(), field));
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public void setIndex(int index) {
+    this.index = index;
+  }
+
+  public String getDataTypeName() {
+    return "ARRAY";
+  }
+
+  Type getArrayOfType(StructField field, DataType dataType) {
+    if (dataType == DataTypes.STRING) {
+      return new ArrayType(VarcharType.VARCHAR);
+    } else if (dataType == DataTypes.INT) {
+      return new ArrayType(IntegerType.INTEGER);
+    } else if (dataType == DataTypes.LONG) {
+      return new ArrayType(BigintType.BIGINT);
+    } else if (dataType == DataTypes.DOUBLE) {
+      return new ArrayType(DoubleType.DOUBLE);
+    } else if (dataType == DataTypes.FLOAT) {
+      return new ArrayType(RealType.REAL);
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return new ArrayType(BooleanType.BOOLEAN);
+    } else {
+      StructField childField = field.getChildren().get(0);
+      return new ArrayType(getArrayOfType(childField, childField.getDataType()));
+    }
+  }
+
+  @Override
+  public Block buildBlock() {
+    return builder.build();
+  }
+
+  public boolean isComplex() {
+    return true;
+  }
+
+  @Override
+  public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override
+  public void putObject(int rowId, Object value) {
+    if (value == null) {
+      putNull(rowId);
+    } else {
+      getChildrenVector().putObject(rowId, value);
+    }
+  }
+
+  public void putArrayObject() {
+    if (this.getType().getName() == "ARRAY") {
+      childBlock = ((ArrayStreamReader) getChildrenVector()).buildBlock();
+    } else if (this.getType().getName() == "STRING") {
+      childBlock = ((SliceStreamReader) getChildrenVector()).buildBlock();
+    } else if (this.getType().getName() == "INT") {

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akkio-97 commented on a change in pull request #3773: [CARBONDATA-3830]Presto complex columns read support

GitBox
In reply to this post by GitBox

akkio-97 commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r462856821



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
##########
@@ -246,7 +241,28 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       vector = ColumnarVectorWrapperDirectFactory
           .getDirectVectorWrapperFactory(vector, vectorInfo.invertedIndex, nullBits, deletedRows,
               true, false);
-      fillVector(pageData, vector, vectorDataType, pageDataType, pageSize, vectorInfo, nullBits);
+      Deque<CarbonColumnVectorImpl> vectorStack = vectorInfo.getVectorStack();
+      // initialize vectorStack if null
+      if (vectorStack == null && vectorInfo.vector.getColumnVector() != null) {
+        vectorStack = new ArrayDeque<CarbonColumnVectorImpl>();
+        vectorStack.push((CarbonColumnVectorImpl) vectorInfo.vector.getColumnVector());
+        vectorInfo.setVectorStack(vectorStack);
+      }
+      /**
+       * if top of vector stack is a complex vector then
+       * add their children into the stack and load them too.
+       * TODO: If there are multiple children push them into stack and load them iteratively
+       */
+      if (vectorStack != null && vectorStack.peek().isComplex()) {

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto complex columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-666245267


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akkio-97 commented on a change in pull request #3773: [CARBONDATA-3830]Presto complex columns read support

GitBox
In reply to this post by GitBox

akkio-97 commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r462856914



##########
File path: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
##########
@@ -176,8 +191,16 @@ private void initBatch() {
     for (int i = 0; i < queryDimension.size(); i++) {
       ProjectionDimension dim = queryDimension.get(i);
       if (dim.getDimension().isComplex()) {
+        List<CarbonDimension> childDimensions =
+                dim.getDimension().getListOfChildDimensions();
+        ArrayList<StructField> childFields = new ArrayList<StructField>();
+        for (int ind = 0; ind < childDimensions.size(); ind++) {
+          ColumnSchema childSchema = childDimensions.get(ind).getColumnSchema();

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
##########
@@ -246,7 +241,28 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       vector = ColumnarVectorWrapperDirectFactory
           .getDirectVectorWrapperFactory(vector, vectorInfo.invertedIndex, nullBits, deletedRows,
               true, false);
-      fillVector(pageData, vector, vectorDataType, pageDataType, pageSize, vectorInfo, nullBits);
+      Deque<CarbonColumnVectorImpl> vectorStack = vectorInfo.getVectorStack();
+      // initialize vectorStack if null
+      if (vectorStack == null && vectorInfo.vector.getColumnVector() != null) {
+        vectorStack = new ArrayDeque<CarbonColumnVectorImpl>();
+        vectorStack.push((CarbonColumnVectorImpl) vectorInfo.vector.getColumnVector());
+        vectorInfo.setVectorStack(vectorStack);
+      }
+      /**

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto complex columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-666245882


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto complex columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-666247824


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akkio-97 commented on a change in pull request #3773: [CARBONDATA-3830]Presto complex columns read support

GitBox
In reply to this post by GitBox

akkio-97 commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r462856821



##########
File path: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
##########
@@ -246,7 +241,28 @@ public void decodeAndFillVector(byte[] pageData, ColumnVectorInfo vectorInfo, Bi
       vector = ColumnarVectorWrapperDirectFactory
           .getDirectVectorWrapperFactory(vector, vectorInfo.invertedIndex, nullBits, deletedRows,
               true, false);
-      fillVector(pageData, vector, vectorDataType, pageDataType, pageSize, vectorInfo, nullBits);
+      Deque<CarbonColumnVectorImpl> vectorStack = vectorInfo.getVectorStack();
+      // initialize vectorStack if null
+      if (vectorStack == null && vectorInfo.vector.getColumnVector() != null) {
+        vectorStack = new ArrayDeque<CarbonColumnVectorImpl>();
+        vectorStack.push((CarbonColumnVectorImpl) vectorInfo.vector.getColumnVector());
+        vectorInfo.setVectorStack(vectorStack);
+      }
+      /**
+       * if top of vector stack is a complex vector then
+       * add their children into the stack and load them too.
+       * TODO: If there are multiple children push them into stack and load them iteratively
+       */
+      if (vectorStack != null && vectorStack.peek().isComplex()) {

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-666397217


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-666397696


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-666769815


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-666770125


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

ajantha-bhat commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-667088775


   @akkio-97 : please also do manual test with huge rows (millions). So that it can cover multiple pages


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-667299231


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-667299327


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

ajantha-bhat commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-668138965


   @akkio-97 : I think this doesn't work with local dictionary enable. Please test it with local dictionary enable


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r465142026



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ArrayStreamReader.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 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.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+
+/**
+ * Class to read the Array Stream
+ */
+
+public class ArrayStreamReader extends CarbonColumnVectorImpl implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+  Block childBlock = null;
+  private int index = 0;
+
+  public ArrayStreamReader(int batchSize, DataType dataType, StructField field) {
+    super(batchSize, dataType);
+    this.batchSize = batchSize;
+    this.type = getArrayOfType(field, dataType);
+    ArrayList<CarbonColumnVectorImpl> childrenList= new ArrayList<>();
+    childrenList.add(CarbonVectorBatch.createDirectStreamReader(this.batchSize, field.getDataType(), field));
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public void setIndex(int index) {
+    this.index = index;
+  }
+
+  public String getDataTypeName() {
+    return "ARRAY";
+  }
+
+  Type getArrayOfType(StructField field, DataType dataType) {
+    if (dataType == DataTypes.STRING) {
+      return new ArrayType(VarcharType.VARCHAR);
+    } else if (dataType == DataTypes.BYTE) {
+      return new ArrayType(TinyintType.TINYINT);
+    } else if (dataType == DataTypes.SHORT) {
+      return new ArrayType(SmallintType.SMALLINT);
+    } else if (dataType == DataTypes.INT) {
+      return new ArrayType(IntegerType.INTEGER);
+    } else if (dataType == DataTypes.LONG) {
+      return new ArrayType(BigintType.BIGINT);
+    } else if (dataType == DataTypes.DOUBLE) {
+      return new ArrayType(DoubleType.DOUBLE);
+    } else if (dataType == DataTypes.FLOAT) {
+      return new ArrayType(RealType.REAL);
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return new ArrayType(BooleanType.BOOLEAN);
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return new ArrayType(TimestampType.TIMESTAMP);
+    } else if (DataTypes.isArrayType(dataType)) {
+      StructField childField = field.getChildren().get(0);
+      return new ArrayType(getArrayOfType(childField, childField.getDataType()));
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override
+  public Block buildBlock() {
+    return builder.build();
+  }
+
+  public boolean isComplex() {
+    return true;
+  }
+
+  @Override
+  public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override
+  public void putObject(int rowId, Object value) {
+    if (value == null) {

Review comment:
       putObject is never used ? Instead of putComplexObject, may be we need to use the same interface




----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r465142026



##########
File path: integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ArrayStreamReader.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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 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.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+
+/**
+ * Class to read the Array Stream
+ */
+
+public class ArrayStreamReader extends CarbonColumnVectorImpl implements PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+  Block childBlock = null;
+  private int index = 0;
+
+  public ArrayStreamReader(int batchSize, DataType dataType, StructField field) {
+    super(batchSize, dataType);
+    this.batchSize = batchSize;
+    this.type = getArrayOfType(field, dataType);
+    ArrayList<CarbonColumnVectorImpl> childrenList= new ArrayList<>();
+    childrenList.add(CarbonVectorBatch.createDirectStreamReader(this.batchSize, field.getDataType(), field));
+    setChildrenVector(childrenList);
+    this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public void setIndex(int index) {
+    this.index = index;
+  }
+
+  public String getDataTypeName() {
+    return "ARRAY";
+  }
+
+  Type getArrayOfType(StructField field, DataType dataType) {
+    if (dataType == DataTypes.STRING) {
+      return new ArrayType(VarcharType.VARCHAR);
+    } else if (dataType == DataTypes.BYTE) {
+      return new ArrayType(TinyintType.TINYINT);
+    } else if (dataType == DataTypes.SHORT) {
+      return new ArrayType(SmallintType.SMALLINT);
+    } else if (dataType == DataTypes.INT) {
+      return new ArrayType(IntegerType.INTEGER);
+    } else if (dataType == DataTypes.LONG) {
+      return new ArrayType(BigintType.BIGINT);
+    } else if (dataType == DataTypes.DOUBLE) {
+      return new ArrayType(DoubleType.DOUBLE);
+    } else if (dataType == DataTypes.FLOAT) {
+      return new ArrayType(RealType.REAL);
+    } else if (dataType == DataTypes.BOOLEAN) {
+      return new ArrayType(BooleanType.BOOLEAN);
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return new ArrayType(TimestampType.TIMESTAMP);
+    } else if (DataTypes.isArrayType(dataType)) {
+      StructField childField = field.getChildren().get(0);
+      return new ArrayType(getArrayOfType(childField, childField.getDataType()));
+    } else {
+      throw new UnsupportedOperationException("Unsupported type: " + dataType);
+    }
+  }
+
+  @Override
+  public Block buildBlock() {
+    return builder.build();
+  }
+
+  public boolean isComplex() {
+    return true;
+  }
+
+  @Override
+  public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override
+  public void putObject(int rowId, Object value) {
+    if (value == null) {

Review comment:
       putObject is never used ? Instead of putArrayObject, may be we need to use the same interface




----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-668849338


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


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-668849764


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


----------------------------------------------------------------
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:
[hidden email]


123456