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 2021/01/07 19:56:37 UTC

[GitHub] [carbondata] akkio-97 opened a new pull request #4073: WIP

akkio-97 opened a new pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073


    ### 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] asfgit closed pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


   


----------------------------------------------------------------
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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -110,8 +110,8 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
     }
 
     @Override
-    public void fillVector(Object valuesToBeConverted, int size,
-        ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
+    public void fillVector(Object valuesToBeConverted, int size, ColumnVectorInfo vectorInfo,

Review comment:
       revert this style change 




----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r561605603



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -328,9 +328,29 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
     public void fillVector(Object valuesToBeConverted, int size,
         ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
       CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
+      int precision;

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -348,6 +368,32 @@ public void fillVector(Object valuesToBeConverted, int size,
             vector.putDecimal(i, value, precision);
           }
         }
+      } else if (valuesToBeConverted instanceof byte[]) {

Review comment:
       done

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
##########
@@ -365,6 +365,15 @@ class SparkStoreCreatorForPresto extends QueryTest with BeforeAndAfterAll{
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE streaming_table""")
   }
 
+  test("Test decimal unscaled converter") {
+    sql("drop table if exists array_decimal")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_decimal (salary array<decimal(20,3)>) STORED AS " +
+      "carbondata"
+    )
+    sql("insert into array_decimal select array(922.580, 3.435) ")

Review comment:
       done

##########
File path: integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoTestUsingSparkStore.scala
##########
@@ -325,4 +326,22 @@ class PrestoTestUsingSparkStore
 
   }
 
+  test("Test decimal unscaled converter") {
+    prestoServer
+      .execute(
+        "create table presto_spark_db.array_decimal(salary array(decimal(20,3)) ) with" +
+        "(format='CARBON') ")
+    copyStoreContents("array_decimal")
+    val result: List[Map[String, Any]] = prestoServer
+      .executeQuery("SELECT * FROM presto_spark_db.array_decimal")
+    assert(result.size == 1)
+    for (i <- 0 to 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] Indhumathi27 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


   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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


   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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: WIP

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


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


----------------------------------------------------------------
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] VenuReddy2103 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
##########
@@ -365,6 +365,15 @@ class SparkStoreCreatorForPresto extends QueryTest with BeforeAndAfterAll{
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE streaming_table""")
   }
 
+  test("Test decimal unscaled converter") {
+    sql("drop table if exists array_decimal")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_decimal (salary array<decimal(20,3)>) STORED AS " +
+      "carbondata"
+    )
+    sql("insert into array_decimal select array(922.580, 3.435) ")

Review comment:
       Didn't drop this table in `afterAll()` like other testcases do ?




----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -328,9 +328,29 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
     public void fillVector(Object valuesToBeConverted, int size,
         ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
       CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
+      int precision;
+      int newMeasureScale;
+      if (vectorInfo.measure == null) {
+        // complex primitive decimal flow comes as dimension
+        precision = ((DecimalType) vector.getType()).getPrecision();
+        newMeasureScale = ((DecimalType) vector.getType()).getScale();
+        size = ColumnVectorInfo.getUpdatedPageSizeForChildVector(vectorInfo, size);
+      } else {
+        precision = vectorInfo.measure.getMeasure().getPrecision();
+        newMeasureScale = vectorInfo.measure.getMeasure().getScale();
+      }

Review comment:
       can this be extracted into a common private method and reuse the same in 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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -109,29 +113,35 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
       return BigDecimal.valueOf((Long) valueToBeConverted, scale);
     }
 
+    private void initializeValues(int size, CarbonColumnVector vector,
+        ColumnVectorInfo vectorInfo) {
+      if (vectorInfo.measure == null) {
+        DecimalType decimalType = (DecimalType) vector.getType();
+        // complex primitive decimal flow comes as dimension
+        this.precision = decimalType.getPrecision();
+        this.newMeasureScale = decimalType.getScale();
+        this.updatedSize = ColumnVectorInfo.getUpdatedPageSizeForChildVector(vectorInfo, size);
+      } else {
+        CarbonMeasure carbonMeasure = vectorInfo.measure.getMeasure();
+        this.precision = carbonMeasure.getPrecision();
+        this.newMeasureScale = carbonMeasure.getScale();
+      }
+    }
+
     @Override
     public void fillVector(Object valuesToBeConverted, int size,
         ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
-      if (!(valuesToBeConverted instanceof byte[])) {
-        throw new UnsupportedOperationException("This object type " + valuesToBeConverted.getClass()
-            + " is not supported in this method");
-      }
       // TODO we need to find way to directly set to vector with out conversion. This way is very
       // inefficient.
       CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      int precision;
-      int newMeasureScale;
-      if (vectorInfo.measure == null) {
-        // complex primitive decimal flow comes as dimension
-        precision = ((DecimalType) vector.getType()).getPrecision();
-        newMeasureScale = ((DecimalType) vector.getType()).getScale();
-        size = ColumnVectorInfo.getUpdatedPageSizeForChildVector(vectorInfo, size);
-      } else {
-        precision = vectorInfo.measure.getMeasure().getPrecision();
-        newMeasureScale = vectorInfo.measure.getMeasure().getScale();
+      initializeValues(size, vector, vectorInfo);

Review comment:
       If used only once, no need to extract common 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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/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:
users@infra.apache.org



[GitHub] [carbondata] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r561606064



##########
File path: integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoTestUsingSparkStore.scala
##########
@@ -325,4 +326,22 @@ class PrestoTestUsingSparkStore
 
   }
 
+  test("Test decimal unscaled converter") {
+    prestoServer
+      .execute(
+        "create table presto_spark_db.array_decimal(salary array(decimal(20,3)) ) with" +
+        "(format='CARBON') ")
+    copyStoreContents("array_decimal")
+    val result: List[Map[String, Any]] = prestoServer
+      .executeQuery("SELECT * FROM presto_spark_db.array_decimal")
+    assert(result.size == 1)
+    for (i <- 0 to 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] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r561605603



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -328,9 +328,29 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
     public void fillVector(Object valuesToBeConverted, int size,
         ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
       CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
+      int precision;

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -348,6 +368,32 @@ public void fillVector(Object valuesToBeConverted, int size,
             vector.putDecimal(i, value, precision);
           }
         }
+      } else if (valuesToBeConverted instanceof byte[]) {

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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r565936260



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -141,100 +165,116 @@ public void fillVector(Object valuesToBeConverted, int size,
         // for string, varchar, binary, date, decimal types
         lengthStoredInBytes = shortSizeInBytes;
       }
-      byte[] data = (byte[]) valuesToBeConverted;
-      if (pageType == DataTypes.BYTE) {
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal.valueOf(data[i], scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      } else if (pageType == DataTypes.SHORT) {
+      if (this instanceof DecimalUnscaledConverter && scale < newMeasureScale) {
+        scale = newMeasureScale;
+      }
+
+      if (valuesToBeConverted instanceof byte[][]) {
+        byte[][] data = (byte[][]) valuesToBeConverted;
         for (int i = 0; i < size; i++) {
           if (nullBitSet.get(i)) {
             vector.putNull(i);
           } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.toShortLittleEndian(data, i * shortSizeInBytes),
-                    scale);
+            BigInteger bigInteger = new BigInteger(data[i]);
+            BigDecimal value = new BigDecimal(bigInteger, scale);
             if (value.scale() < newMeasureScale) {
               value = value.setScale(newMeasureScale);
             }
             vector.putDecimal(i, value, precision);
           }
         }
-      } else if (pageType == DataTypes.SHORT_INT) {
-        int shortIntSizeInBytes = DataTypes.SHORT_INT.getSizeInBytes();
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.valueOf3Bytes(data, i * shortIntSizeInBytes),
-                    scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
+      } else if (valuesToBeConverted instanceof byte[]) {
+        byte[] data = (byte[]) valuesToBeConverted;
+        if (pageType == DataTypes.BYTE) {
+          for (int i = 0; i < size; i++) {
+            if (nullBitSet.get(i)) {
+              vector.putNull(i);
+            } else {
+              BigDecimal value = BigDecimal.valueOf(data[i], scale);

Review comment:
       done

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
##########
@@ -365,6 +366,15 @@ class SparkStoreCreatorForPresto extends QueryTest with BeforeAndAfterAll{
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE streaming_table""")
   }
 
+  test("Test decimal unscaled converter") {
+    sql("drop table if exists array_decimal")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_decimal (salary array<decimal(20,3)>) STORED AS " +

Review comment:
       okay

##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -88,11 +88,34 @@ void fillVector(Object valuesToBeConverted, int size, ColumnVectorInfo info, Bit
 
     DecimalConverterType getDecimalConverterType();
 
+    default int getPrecisionForDimension(CarbonColumnVector vector) {

Review comment:
       okay

##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -328,9 +328,29 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
     public void fillVector(Object valuesToBeConverted, int size,
         ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
       CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
+      int precision;
+      int newMeasureScale;
+      if (vectorInfo.measure == null) {
+        // complex primitive decimal flow comes as dimension
+        precision = ((DecimalType) vector.getType()).getPrecision();
+        newMeasureScale = ((DecimalType) vector.getType()).getScale();
+        size = ColumnVectorInfo.getUpdatedPageSizeForChildVector(vectorInfo, size);
+      } else {
+        precision = vectorInfo.measure.getMeasure().getPrecision();
+        newMeasureScale = vectorInfo.measure.getMeasure().getScale();
+      }

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] Indhumathi27 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -88,11 +88,34 @@ void fillVector(Object valuesToBeConverted, int size, ColumnVectorInfo info, Bit
 
     DecimalConverterType getDecimalConverterType();
 
+    default int getPrecisionForDimension(CarbonColumnVector vector) {

Review comment:
       I can see, these methods are called from a single place only. Then, why it is required to create these methods ?. Instead, modify caller method like below:
   """
   public void initializeValues(int size, CarbonColumnVector vector, ColumnVectorInfo vectorInfo) {
         if (vectorInfo.measure == null) {
           DecimalType decimalType = (DecimalType) vector.getType();
           // complex primitive decimal flow comes as dimension
           this.precision = decimalType.getPrecision();
           this.newMeasureScale = decimalType.getScale();
           this.updatedSize = ColumnVectorInfo.getUpdatedPageSizeForChildVector(vectorInfo, size);
         } else {
           CarbonMeasure carbonMeasure = vectorInfo.measure.getMeasure();
           this.precision = carbonMeasure.getPrecision();
           this.newMeasureScale = carbonMeasure.getScale();
         }
       } """




----------------------------------------------------------------
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] Indhumathi27 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -141,100 +165,116 @@ public void fillVector(Object valuesToBeConverted, int size,
         // for string, varchar, binary, date, decimal types
         lengthStoredInBytes = shortSizeInBytes;
       }
-      byte[] data = (byte[]) valuesToBeConverted;
-      if (pageType == DataTypes.BYTE) {
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal.valueOf(data[i], scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      } else if (pageType == DataTypes.SHORT) {
+      if (this instanceof DecimalUnscaledConverter && scale < newMeasureScale) {
+        scale = newMeasureScale;
+      }
+
+      if (valuesToBeConverted instanceof byte[][]) {
+        byte[][] data = (byte[][]) valuesToBeConverted;
         for (int i = 0; i < size; i++) {
           if (nullBitSet.get(i)) {
             vector.putNull(i);
           } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.toShortLittleEndian(data, i * shortSizeInBytes),
-                    scale);
+            BigInteger bigInteger = new BigInteger(data[i]);
+            BigDecimal value = new BigDecimal(bigInteger, scale);
             if (value.scale() < newMeasureScale) {
               value = value.setScale(newMeasureScale);
             }
             vector.putDecimal(i, value, precision);
           }
         }
-      } else if (pageType == DataTypes.SHORT_INT) {
-        int shortIntSizeInBytes = DataTypes.SHORT_INT.getSizeInBytes();
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.valueOf3Bytes(data, i * shortIntSizeInBytes),
-                    scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
+      } else if (valuesToBeConverted instanceof byte[]) {
+        byte[] data = (byte[]) valuesToBeConverted;
+        if (pageType == DataTypes.BYTE) {
+          for (int i = 0; i < size; i++) {
+            if (nullBitSet.get(i)) {
+              vector.putNull(i);
+            } else {
+              BigDecimal value = BigDecimal.valueOf(data[i], scale);

Review comment:
       i think, it will not lead to NPE, if all cases are handled. Please check




----------------------------------------------------------------
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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -327,25 +287,28 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
     @Override
     public void fillVector(Object valuesToBeConverted, int size,
         ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
-      CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
-      if (scale < newMeasureScale) {
-        scale = newMeasureScale;
-      }
-      if (valuesToBeConverted instanceof byte[][]) {
-        byte[][] data = (byte[][]) valuesToBeConverted;
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigInteger bigInteger = new BigInteger(data[i]);
-            BigDecimal value = new BigDecimal(bigInteger, scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
+      if (valuesToBeConverted instanceof byte[]) {
+        super.fillVector(valuesToBeConverted, size, vectorInfo, nullBitSet, pageType);

Review comment:
       could have merged into single statement. no need to modify now.  it's enough




----------------------------------------------------------------
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] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r566734422



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -324,33 +311,6 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
       return new BigDecimal(bigInteger, scale);
     }
 
-    @Override
-    public void fillVector(Object valuesToBeConverted, int size,
-        ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
-      CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
-      if (scale < newMeasureScale) {
-        scale = newMeasureScale;
-      }
-      if (valuesToBeConverted instanceof byte[][]) {
-        byte[][] data = (byte[][]) valuesToBeConverted;
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigInteger bigInteger = new BigInteger(data[i]);
-            BigDecimal value = new BigDecimal(bigInteger, scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      }

Review comment:
       okay




----------------------------------------------------------------
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] Indhumathi27 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -141,100 +165,116 @@ public void fillVector(Object valuesToBeConverted, int size,
         // for string, varchar, binary, date, decimal types
         lengthStoredInBytes = shortSizeInBytes;
       }
-      byte[] data = (byte[]) valuesToBeConverted;
-      if (pageType == DataTypes.BYTE) {
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal.valueOf(data[i], scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      } else if (pageType == DataTypes.SHORT) {
+      if (this instanceof DecimalUnscaledConverter && scale < newMeasureScale) {
+        scale = newMeasureScale;
+      }
+
+      if (valuesToBeConverted instanceof byte[][]) {
+        byte[][] data = (byte[][]) valuesToBeConverted;
         for (int i = 0; i < size; i++) {
           if (nullBitSet.get(i)) {
             vector.putNull(i);
           } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.toShortLittleEndian(data, i * shortSizeInBytes),
-                    scale);
+            BigInteger bigInteger = new BigInteger(data[i]);
+            BigDecimal value = new BigDecimal(bigInteger, scale);
             if (value.scale() < newMeasureScale) {
               value = value.setScale(newMeasureScale);
             }
             vector.putDecimal(i, value, precision);
           }
         }
-      } else if (pageType == DataTypes.SHORT_INT) {
-        int shortIntSizeInBytes = DataTypes.SHORT_INT.getSizeInBytes();
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.valueOf3Bytes(data, i * shortIntSizeInBytes),
-                    scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
+      } else if (valuesToBeConverted instanceof byte[]) {
+        byte[] data = (byte[]) valuesToBeConverted;
+        if (pageType == DataTypes.BYTE) {
+          for (int i = 0; i < size; i++) {
+            if (nullBitSet.get(i)) {
+              vector.putNull(i);
+            } else {
+              BigDecimal value = BigDecimal.valueOf(data[i], scale);

Review comment:
       I think, we can still refactor the code for Cases: Byte, Short, Short_Int, Int and Long, As only difference is getting Bigdecimal value based on page data type




----------------------------------------------------------------
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] Indhumathi27 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
##########
@@ -365,6 +366,15 @@ class SparkStoreCreatorForPresto extends QueryTest with BeforeAndAfterAll{
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE streaming_table""")
   }
 
+  test("Test decimal unscaled converter") {
+    sql("drop table if exists array_decimal")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_decimal (salary array<decimal(20,3)>) STORED AS " +

Review comment:
       can you add a struct column with decimal type 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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -324,33 +311,6 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
       return new BigDecimal(bigInteger, scale);
     }
 
-    @Override
-    public void fillVector(Object valuesToBeConverted, int size,
-        ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
-      CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
-      if (scale < newMeasureScale) {
-        scale = newMeasureScale;
-      }
-      if (valuesToBeConverted instanceof byte[][]) {
-        byte[][] data = (byte[][]) valuesToBeConverted;
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigInteger bigInteger = new BigInteger(data[i]);
-            BigDecimal value = new BigDecimal(bigInteger, scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      }

Review comment:
       add an else check and call the super.fillvector if it is not byte[][], instead of copying this code in super class and changing its implementation. super class was meant only for one dimensional byte[]




----------------------------------------------------------------
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] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r561605668



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
##########
@@ -365,6 +365,15 @@ class SparkStoreCreatorForPresto extends QueryTest with BeforeAndAfterAll{
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE streaming_table""")
   }
 
+  test("Test decimal unscaled converter") {
+    sql("drop table if exists array_decimal")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_decimal (salary array<decimal(20,3)>) STORED AS " +
+      "carbondata"
+    )
+    sql("insert into array_decimal select array(922.580, 3.435) ")

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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] Indhumathi27 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -141,100 +165,116 @@ public void fillVector(Object valuesToBeConverted, int size,
         // for string, varchar, binary, date, decimal types
         lengthStoredInBytes = shortSizeInBytes;
       }
-      byte[] data = (byte[]) valuesToBeConverted;
-      if (pageType == DataTypes.BYTE) {
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal.valueOf(data[i], scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      } else if (pageType == DataTypes.SHORT) {
+      if (this instanceof DecimalUnscaledConverter && scale < newMeasureScale) {
+        scale = newMeasureScale;
+      }
+
+      if (valuesToBeConverted instanceof byte[][]) {
+        byte[][] data = (byte[][]) valuesToBeConverted;
         for (int i = 0; i < size; i++) {
           if (nullBitSet.get(i)) {
             vector.putNull(i);
           } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.toShortLittleEndian(data, i * shortSizeInBytes),
-                    scale);
+            BigInteger bigInteger = new BigInteger(data[i]);
+            BigDecimal value = new BigDecimal(bigInteger, scale);
             if (value.scale() < newMeasureScale) {
               value = value.setScale(newMeasureScale);
             }
             vector.putDecimal(i, value, precision);
           }
         }
-      } else if (pageType == DataTypes.SHORT_INT) {
-        int shortIntSizeInBytes = DataTypes.SHORT_INT.getSizeInBytes();
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.valueOf3Bytes(data, i * shortIntSizeInBytes),
-                    scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
+      } else if (valuesToBeConverted instanceof byte[]) {
+        byte[] data = (byte[]) valuesToBeConverted;
+        if (pageType == DataTypes.BYTE) {
+          for (int i = 0; i < size; i++) {
+            if (nullBitSet.get(i)) {
+              vector.putNull(i);
+            } else {
+              BigDecimal value = BigDecimal.valueOf(data[i], scale);

Review comment:
       I think, we can still refactor the code for Cases: Byte, Short, Short_Int, Int and Long, As only difference is getting Bigdecimal value.




----------------------------------------------------------------
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] Indhumathi27 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
##########
@@ -365,6 +366,15 @@ class SparkStoreCreatorForPresto extends QueryTest with BeforeAndAfterAll{
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE streaming_table""")
   }
 
+  test("Test decimal unscaled converter") {
+    sql("drop table if exists array_decimal")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_decimal (salary array<decimal(20,3)>) STORED AS " +

Review comment:
       better to add, as it is a new scenario without testcase




----------------------------------------------------------------
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] VenuReddy2103 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoTestUsingSparkStore.scala
##########
@@ -325,4 +326,22 @@ class PrestoTestUsingSparkStore
 
   }
 
+  test("Test decimal unscaled converter") {
+    prestoServer
+      .execute(
+        "create table presto_spark_db.array_decimal(salary array(decimal(20,3)) ) with" +
+        "(format='CARBON') ")
+    copyStoreContents("array_decimal")
+    val result: List[Map[String, Any]] = prestoServer
+      .executeQuery("SELECT * FROM presto_spark_db.array_decimal")
+    assert(result.size == 1)
+    for (i <- 0 to 0) {

Review comment:
       result size is only 1. Suggest to remove this redundant loop if 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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r565258047



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
##########
@@ -365,6 +366,15 @@ class SparkStoreCreatorForPresto extends QueryTest with BeforeAndAfterAll{
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE streaming_table""")
   }
 
+  test("Test decimal unscaled converter") {
+    sql("drop table if exists array_decimal")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_decimal (salary array<decimal(20,3)>) STORED AS " +

Review comment:
       can add but it is of no use as the flow will be common for both array and struct.




----------------------------------------------------------------
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] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r565244018



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -141,100 +165,116 @@ public void fillVector(Object valuesToBeConverted, int size,
         // for string, varchar, binary, date, decimal types
         lengthStoredInBytes = shortSizeInBytes;
       }
-      byte[] data = (byte[]) valuesToBeConverted;
-      if (pageType == DataTypes.BYTE) {
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal.valueOf(data[i], scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      } else if (pageType == DataTypes.SHORT) {
+      if (this instanceof DecimalUnscaledConverter && scale < newMeasureScale) {
+        scale = newMeasureScale;
+      }
+
+      if (valuesToBeConverted instanceof byte[][]) {
+        byte[][] data = (byte[][]) valuesToBeConverted;
         for (int i = 0; i < size; i++) {
           if (nullBitSet.get(i)) {
             vector.putNull(i);
           } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.toShortLittleEndian(data, i * shortSizeInBytes),
-                    scale);
+            BigInteger bigInteger = new BigInteger(data[i]);
+            BigDecimal value = new BigDecimal(bigInteger, scale);
             if (value.scale() < newMeasureScale) {
               value = value.setScale(newMeasureScale);
             }
             vector.putDecimal(i, value, precision);
           }
         }
-      } else if (pageType == DataTypes.SHORT_INT) {
-        int shortIntSizeInBytes = DataTypes.SHORT_INT.getSizeInBytes();
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.valueOf3Bytes(data, i * shortIntSizeInBytes),
-                    scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
+      } else if (valuesToBeConverted instanceof byte[]) {
+        byte[] data = (byte[]) valuesToBeConverted;
+        if (pageType == DataTypes.BYTE) {
+          for (int i = 0; i < size; i++) {
+            if (nullBitSet.get(i)) {
+              vector.putNull(i);
+            } else {
+              BigDecimal value = BigDecimal.valueOf(data[i], scale);

Review comment:
       Doing that might lead to NPE if Bigdecimal value is not initialized in any of the mentioned cases




----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] VenuReddy2103 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -348,6 +368,32 @@ public void fillVector(Object valuesToBeConverted, int size,
             vector.putDecimal(i, value, precision);
           }
         }
+      } else if (valuesToBeConverted instanceof byte[]) {

Review comment:
       Shall we avoid this code and reuse ? Same as above.




----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -348,6 +368,32 @@ public void fillVector(Object valuesToBeConverted, int size,
             vector.putDecimal(i, value, precision);
           }
         }
+      } else if (valuesToBeConverted instanceof byte[]) {

Review comment:
       yes, check and resue the code for this 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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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






----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -327,25 +287,28 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
     @Override
     public void fillVector(Object valuesToBeConverted, int size,
         ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
-      CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
-      if (scale < newMeasureScale) {
-        scale = newMeasureScale;
-      }
-      if (valuesToBeConverted instanceof byte[][]) {
-        byte[][] data = (byte[][]) valuesToBeConverted;
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigInteger bigInteger = new BigInteger(data[i]);
-            BigDecimal value = new BigDecimal(bigInteger, scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
+      if (valuesToBeConverted instanceof byte[]) {
+        super.fillVector(valuesToBeConverted, size, vectorInfo, nullBitSet, pageType);

Review comment:
       use return and remove else.




----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] akkio-97 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

Posted by GitBox <gi...@apache.org>.
akkio-97 commented on a change in pull request #4073:
URL: https://github.com/apache/carbondata/pull/4073#discussion_r565258047



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
##########
@@ -365,6 +366,15 @@ class SparkStoreCreatorForPresto extends QueryTest with BeforeAndAfterAll{
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE streaming_table""")
   }
 
+  test("Test decimal unscaled converter") {
+    sql("drop table if exists array_decimal")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_decimal (salary array<decimal(20,3)>) STORED AS " +

Review comment:
       can add but it is of no use as the flow will be common for both




----------------------------------------------------------------
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] VenuReddy2103 commented on a change in pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -328,9 +328,29 @@ public BigDecimal getDecimal(Object valueToBeConverted) {
     public void fillVector(Object valuesToBeConverted, int size,
         ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
       CarbonColumnVector vector = getCarbonColumnVector(vectorInfo, nullBitSet);
-      //TODO handle complex child
-      int precision = vectorInfo.measure.getMeasure().getPrecision();
-      int newMeasureScale = vectorInfo.measure.getMeasure().getScale();
+      int precision;

Review comment:
       These changes seem to be same as in`DecimalIntConverter.fillVector()`. Shall we reuse it instead of duplicating same logic again here ? Probably `DecimalUnscaledConverter` can extend `DecimalIntConverter` and overide fillVector like below ?
   
   > public void fillVector(Object valuesToBeConverted, int size, ColumnVectorInfo vectorInfo, BitSet nullBitSet, DataType pageType) {
         if (valuesToBeConverted instanceof byte[][]) {
           ...
         } else {
           super.fillVector(valuesToBeConverted, size, vectorInfo, nullBitSet, pageType);
         }
       }




----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: WIP

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


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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


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


----------------------------------------------------------------
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 #4073: [CARBONDATA-4104] Vector filling for complex decimal type needs to be handled

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



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
##########
@@ -141,72 +151,48 @@ public void fillVector(Object valuesToBeConverted, int size,
         // for string, varchar, binary, date, decimal types
         lengthStoredInBytes = shortSizeInBytes;
       }
-      byte[] data = (byte[]) valuesToBeConverted;
-      if (pageType == DataTypes.BYTE) {
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal.valueOf(data[i], scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      } else if (pageType == DataTypes.SHORT) {
-        for (int i = 0; i < size; i++) {
-          if (nullBitSet.get(i)) {
-            vector.putNull(i);
-          } else {
-            BigDecimal value = BigDecimal
-                .valueOf(ByteUtil.toShortLittleEndian(data, i * shortSizeInBytes),
-                    scale);
-            if (value.scale() < newMeasureScale) {
-              value = value.setScale(newMeasureScale);
-            }
-            vector.putDecimal(i, value, precision);
-          }
-        }
-      } else if (pageType == DataTypes.SHORT_INT) {
-        int shortIntSizeInBytes = DataTypes.SHORT_INT.getSizeInBytes();
+      if (this instanceof DecimalUnscaledConverter && scale < newMeasureScale) {
+        scale = newMeasureScale;
+      }
+      if (valuesToBeConverted instanceof byte[][]) {

Review comment:
       revert this change as discussed.




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