This is an automated email from the ASF dual-hosted git repository.

ajantha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git


The following commit(s) were added to refs/heads/master by this push:
     new 46a46a0  [CARBONDATA-4109] Improve carbondata coverage for 
presto-integration code
46a46a0 is described below

commit 46a46a0d369d5f1cc82cf777d68eea9c45a5c210
Author: akkio-97 <[email protected]>
AuthorDate: Mon Jan 11 22:14:04 2021 +0530

    [CARBONDATA-4109] Improve carbondata coverage for presto-integration code
    
    Why is this PR needed?
    Few scenarios had missing coverage in presto-integration code. This PR aims 
to improve it by considering all such scenarios.
    
    Dead code- ObjectStreamReader.java was created with an aim to query complex 
types. Instead ComplexTypeStreamReader was created. Making ObjectStreamreader 
obsolete.
    
    What changes were proposed in this PR?
    Test cases added for scenarios that were not covered earlier in 
presto-integration code
    Removed dead code.
    
    Does this PR introduce any user interface change?
    No
    
    Is any new testcase added?
    Yes
    
    This closes #4074
---
 .../carbondata/presto/CarbonVectorBatch.java       |  2 +-
 .../presto/impl/CarbonLocalInputSplit.java         | 19 -----
 .../presto/impl/CarbonLocalMultiBlockSplit.java    |  7 --
 .../apache/carbondata/presto/PrestoFilterUtil.java | 24 ------
 .../presto/readers/ObjectStreamReader.java         | 77 -------------------
 .../carbondata/presto/server/PrestoTestUtil.scala  | 75 +++++++++++++++++++
 .../carbondata/presto/server/PrestoTestUtil.scala  | 74 +++++++++++++++++++
 .../PrestoTestUsingSparkStore.scala                | 78 +++++++++++++++++---
 .../dataload/SparkStoreCreatorForPresto.scala      | 86 +++++++++++++++++++++-
 9 files changed, 304 insertions(+), 138 deletions(-)

diff --git 
a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java
 
b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java
index 88c18a7..2b1c2e2 100644
--- 
a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java
+++ 
b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java
@@ -93,7 +93,7 @@ public class CarbonVectorBatch {
     } else if (field.getDataType().isComplexType()) {
       return new ComplexTypeStreamReader(batchSize, field);
     } else {
-      return new ObjectStreamReader(batchSize, field.getDataType());
+      throw new UnsupportedOperationException("Datatype is invalid");
     }
   }
 
diff --git 
a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalInputSplit.java
 
b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalInputSplit.java
index 1b934f4..e0c9fb8 100755
--- 
a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalInputSplit.java
+++ 
b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalInputSplit.java
@@ -47,7 +47,6 @@ public class CarbonLocalInputSplit {
   private String[] deleteDeltaFiles;
   private String blockletId;
   private String detailInfo;
-  private int fileFormatOrdinal;
   private FileFormat fileFormat;
 
   /**
@@ -96,30 +95,14 @@ public class CarbonLocalInputSplit {
   }
 
   @JsonProperty
-  public String getDetailInfo() {
-    return detailInfo;
-  }
-
-  @JsonProperty
   public String getBlockletId() {
     return blockletId;
   }
 
-  @JsonProperty
-  public int getFileFormatOrdinal() {
-    return fileFormatOrdinal;
-  }
-
   public FileFormat getFileFormat() {
     return fileFormat;
   }
 
-  public void setDetailInfo(BlockletDetailInfo blockletDetailInfo) {
-    Gson gson = new Gson();
-    detailInfo = gson.toJson(blockletDetailInfo);
-
-  }
-
   @JsonCreator
   public CarbonLocalInputSplit(@JsonProperty("segmentId") String segmentId,
       @JsonProperty("path") String path, @JsonProperty("start") long start,
@@ -137,12 +120,10 @@ public class CarbonLocalInputSplit {
     this.segmentId = segmentId;
     this.locations = locations;
     this.numberOfBlocklets = numberOfBlocklets;
-    //this.tableBlockInfo = tableBlockInfo;
     this.version = version;
     this.deleteDeltaFiles = deleteDeltaFiles;
     this.blockletId = blockletId;
     this.detailInfo = detailInfo;
-    this.fileFormatOrdinal = fileFormatOrdinal;
     this.fileFormat = FileFormat.getByOrdinal(fileFormatOrdinal);
   }
 
diff --git 
a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalMultiBlockSplit.java
 
b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalMultiBlockSplit.java
index 6a5e005..a363fd5 100755
--- 
a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalMultiBlockSplit.java
+++ 
b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonLocalMultiBlockSplit.java
@@ -47,13 +47,6 @@ public class CarbonLocalMultiBlockSplit {
 
   private FileFormat fileFormat = FileFormat.COLUMNAR_V3;
 
-  private long length;
-
-  @JsonProperty
-  public long getLength() {
-    return length;
-  }
-
   @JsonProperty
   public String[] getLocations() {
     return locations;
diff --git 
a/integration/presto/src/main/prestosql/org/apache/carbondata/presto/PrestoFilterUtil.java
 
b/integration/presto/src/main/prestosql/org/apache/carbondata/presto/PrestoFilterUtil.java
index 45e7158..2e9cd48 100755
--- 
a/integration/presto/src/main/prestosql/org/apache/carbondata/presto/PrestoFilterUtil.java
+++ 
b/integration/presto/src/main/prestosql/org/apache/carbondata/presto/PrestoFilterUtil.java
@@ -19,21 +19,15 @@ package org.apache.carbondata.presto;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
-import static java.lang.String.format;
 import static java.util.stream.Collectors.toList;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
@@ -50,26 +44,19 @@ import 
org.apache.carbondata.core.scan.expression.logical.OrExpression;
 import io.airlift.slice.Slice;
 import io.prestosql.plugin.hive.HiveColumnHandle;
 import io.prestosql.plugin.hive.HiveType;
-import io.prestosql.spi.PrestoException;
 import io.prestosql.spi.predicate.Domain;
 import io.prestosql.spi.predicate.Range;
 import io.prestosql.spi.predicate.TupleDomain;
 import io.prestosql.spi.type.Decimals;
-import io.prestosql.spi.type.Type;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED;
 
 /**
  * PrestoFilterUtil create the carbonData Expression from the presto-domain
  */
 public class PrestoFilterUtil {
 
-  private static Map<Integer, Expression> filterMap = new HashMap<>();
-
-  private static final String HIVE_DEFAULT_DYNAMIC_PARTITION = 
"__HIVE_DEFAULT_PARTITION__";
-
   /**
    * @param columnHandle
    * @return
@@ -250,15 +237,4 @@ public class PrestoFilterUtil {
 
     return rawData;
   }
-
-  /**
-   * get the filters from key
-   */
-  static Expression getFilters(Integer key) {
-    return filterMap.get(key);
-  }
-
-  static void setFilter(Integer tableId, Expression filter) {
-    filterMap.put(tableId, filter);
-  }
 }
\ No newline at end of file
diff --git 
a/integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ObjectStreamReader.java
 
b/integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ObjectStreamReader.java
deleted file mode 100644
index 5855f57..0000000
--- 
a/integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ObjectStreamReader.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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 org.apache.carbondata.core.metadata.datatype.DataType;
-import 
org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
-
-import io.prestosql.spi.block.Block;
-import io.prestosql.spi.block.BlockBuilder;
-import io.prestosql.spi.type.IntegerType;
-import io.prestosql.spi.type.Type;
-
-/**
- * Class to read the Object Stream
- */
-public class ObjectStreamReader extends CarbonColumnVectorImpl implements 
PrestoVectorBlockBuilder {
-
-  protected int batchSize;
-
-  protected Type type = IntegerType.INTEGER;
-
-  protected BlockBuilder builder;
-
-  public ObjectStreamReader(int batchSize, DataType dataType) {
-    super(batchSize, dataType);
-    this.batchSize = batchSize;
-    this.builder = type.createBlockBuilder(null, batchSize);
-  }
-
-  @Override
-  public Block buildBlock() {
-    return builder.build();
-  }
-
-  @Override
-  public void setBatchSize(int batchSize) {
-    this.batchSize = batchSize;
-  }
-
-  @Override
-  public void putObject(int rowId, Object value) {
-    type.writeObject(builder, value);
-  }
-
-  @Override
-  public void putNull(int rowId) {
-    builder.appendNull();
-  }
-
-  @Override
-  public void reset() {
-    builder = type.createBlockBuilder(null, batchSize);
-  }
-
-  @Override
-  public void putNulls(int rowId, int count) {
-    for (int i = 0; i < count; i++) {
-      builder.appendNull();
-    }
-  }
-
-}
diff --git 
a/integration/presto/src/test/prestodb/org/apache/carbondata/presto/server/PrestoTestUtil.scala
 
b/integration/presto/src/test/prestodb/org/apache/carbondata/presto/server/PrestoTestUtil.scala
index 646d191..9ca66d5 100644
--- 
a/integration/presto/src/test/prestodb/org/apache/carbondata/presto/server/PrestoTestUtil.scala
+++ 
b/integration/presto/src/test/prestodb/org/apache/carbondata/presto/server/PrestoTestUtil.scala
@@ -188,4 +188,79 @@ object PrestoTestUtil {
     assert(data1.sameElements(Array("India", "China", "Japan")))
     assert(data2.sameElements(Array("Korea")))
   }
+
+
+  // this method depends on prestodb jdbc PrestoArray class
+  def validateShortData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 0) {
+      val data = result(i)("salary")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array(4352, 35, 3)))
+    }
+  }
+
+  // this method depends on prestodb jdbc PrestoArray class
+  def validateIntData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 1) {
+      val data = result(i)("salary")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array(21474836, 21474839, 23, 3)) ||
+             data.sameElements(Array(21474836, 21474839)))
+    }
+  }
+
+  // this method depends on prestodb jdbc PrestoArray class
+  def validateDoubleData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 5) {
+      val data = result(i)("salary")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array(2, 3, 4)) ||
+             data.sameElements(Array(242, 35, 43)) ||
+             data.sameElements(Array(32799, 32767)) ||
+             data.sameElements(Array(21546546, 32546546, 43211564)) ||
+             data.sameElements(Array(327.99, 3.2799)) ||
+             data.sameElements(Array(1, 2345, 108787.123)))
+    }
+  }
+
+  // this method depends on prestodb jdbc PrestoArray class
+  def validateLongData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 2) {
+      val data = result(i)("salary")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array(215, 23, 3)) ||
+             data.sameElements(Array(32800, 23, 3)) ||
+             data.sameElements(Array(32800, 214748364, 3))
+      )
+    }
+  }
+
+  // this method depends on prestodb jdbc PrestoArray class
+  def validateTimestampData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 1) {
+      val data = result(i)("time")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array("2020-01-11 12:00:45.000", "2020-01-11 
12:01:45.000")) ||
+             data.sameElements(Array("2020-01-10 12:30:45.000", "2015-01-11 
12:01:45.000")))
+    }
+  }
+
+  // this method depends on prestodb jdbc PrestoArray class
+  def validateDecimalData(result: List[Map[String, Any]]): Unit = {
+    val data = result(0)("salary")
+      .asInstanceOf[PrestoArray]
+      .getArray()
+      .asInstanceOf[Array[Object]]
+    assert(data.sameElements(Array("922.580", "3.435")))
+  }
 }
diff --git 
a/integration/presto/src/test/prestosql/org/apache/carbondata/presto/server/PrestoTestUtil.scala
 
b/integration/presto/src/test/prestosql/org/apache/carbondata/presto/server/PrestoTestUtil.scala
index 9044933..daa8dc8 100644
--- 
a/integration/presto/src/test/prestosql/org/apache/carbondata/presto/server/PrestoTestUtil.scala
+++ 
b/integration/presto/src/test/prestosql/org/apache/carbondata/presto/server/PrestoTestUtil.scala
@@ -189,4 +189,78 @@ object PrestoTestUtil {
       assert(data2.sameElements(Array("Korea")))
     }
 
+  // this method depends on prestosql jdbc PrestoArray class
+  def validateShortData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 0) {
+      val data = result(i)("salary")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array(4352, 35, 3)))
+    }
+  }
+
+  // this method depends on prestosql jdbc PrestoArray class
+  def validateIntData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 1) {
+      val data = result(i)("salary")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array(21474836, 21474839, 23, 3)) ||
+             data.sameElements(Array(21474836, 21474839)))
+    }
+  }
+
+  // this method depends on prestosql jdbc PrestoArray class
+  def validateDoubleData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 5) {
+      val data = result(i)("salary")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array(2, 3, 4)) ||
+             data.sameElements(Array(242, 35, 43)) ||
+             data.sameElements(Array(32799, 32767)) ||
+             data.sameElements(Array(21546546, 32546546, 43211564)) ||
+             data.sameElements(Array(327.99, 3.2799)) ||
+             data.sameElements(Array(1, 2345, 108787.123)))
+    }
+  }
+
+  // this method depends on prestosql jdbc PrestoArray class
+  def validateLongData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 2) {
+      val data = result(i)("salary")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array(215, 23, 3)) ||
+             data.sameElements(Array(32800, 23, 3)) ||
+             data.sameElements(Array(32800, 214748364, 3))
+      )
+    }
+  }
+
+  // this method depends on prestosql jdbc PrestoArray class
+  def validateTimestampData(result: List[Map[String, Any]]): Unit = {
+    for (i <- 0 to 1) {
+      val data = result(i)("time")
+        .asInstanceOf[PrestoArray]
+        .getArray()
+        .asInstanceOf[Array[Object]]
+      assert(data.sameElements(Array("2020-01-11 12:00:45.000", "2020-01-11 
12:01:45.000")) ||
+             data.sameElements(Array("2020-01-10 12:30:45.000", "2015-01-11 
12:01:45.000")))
+    }
+  }
+
+  // this method depends on prestosql jdbc PrestoArray class
+  def validateDecimalData(result: List[Map[String, Any]]): Unit = {
+    val data = result(0)("salary")
+      .asInstanceOf[PrestoArray]
+      .getArray()
+      .asInstanceOf[Array[Object]]
+    assert(data.sameElements(Array("922.580", "3.435")))
+  }
+
 }
diff --git 
a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoTestUsingSparkStore.scala
 
b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoTestUsingSparkStore.scala
index 2e44dd7..67f2617 100644
--- 
a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoTestUsingSparkStore.scala
+++ 
b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoTestUsingSparkStore.scala
@@ -20,7 +20,6 @@ package org.apache.carbondata.presto.integrationtest
 import java.io.{File}
 import java.util
 
-import io.prestosql.jdbc.PrestoArray
 import org.apache.commons.io.FileUtils
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuiteLike}
 
@@ -28,7 +27,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.presto.server.{PrestoServer}
+import org.apache.carbondata.presto.server.{PrestoServer, PrestoTestUtil}
 
 class PrestoTestUsingSparkStore
   extends FunSuiteLike with BeforeAndAfterAll with BeforeAndAfterEach {
@@ -57,8 +56,8 @@ class PrestoTestUsingSparkStore
   override def afterAll(): Unit = {
     prestoServer.stopServer()
     CarbonUtil.deleteFoldersAndFiles(FileFactory.getCarbonFile(storePath))
-     CarbonUtil.deleteFoldersAndFiles(FileFactory.getCarbonFile
-     (s"$sparkStorePath"))
+    CarbonUtil.deleteFoldersAndFiles(FileFactory.getCarbonFile
+    (s"$sparkStorePath"))
   }
 
   def copyStoreContents(tableName: String): Any = {
@@ -305,6 +304,71 @@ class PrestoTestUsingSparkStore
     }
   }
 
+  test("Test short vector datatype") {
+    prestoServer.execute("drop table if exists presto_spark_db.array_short")
+    prestoServer
+      .execute(
+        "create table presto_spark_db.array_short(salary array(smallint) ) 
with" +
+        "(format='CARBON') ")
+    copyStoreContents("array_short")
+    val result: List[Map[String, Any]] = prestoServer
+      .executeQuery("SELECT * FROM presto_spark_db.array_short")
+    assert(result.size == 1)
+    PrestoTestUtil.validateShortData(result)
+  }
+
+  test("Test int vector datatype") {
+    prestoServer.execute("drop table if exists presto_spark_db.array_int")
+    prestoServer
+      .execute(
+        "create table presto_spark_db.array_int(salary array(int) ) with" +
+        "(format='CARBON') ")
+    copyStoreContents("array_int")
+    val result: List[Map[String, Any]] = prestoServer
+      .executeQuery("SELECT * FROM presto_spark_db.array_int")
+    assert(result.size == 2)
+    PrestoTestUtil.validateIntData(result)
+  }
+
+  test("Test double vector datatype") {
+    prestoServer.execute("drop table if exists presto_spark_db.array_double")
+    prestoServer
+      .execute(
+        "create table presto_spark_db.array_double(salary array(double) ) 
with" +
+        "(format='CARBON') ")
+    copyStoreContents("array_double")
+    val result: List[Map[String, Any]] = prestoServer
+      .executeQuery("SELECT * FROM presto_spark_db.array_double")
+    assert(result.size == 6)
+    PrestoTestUtil.validateDoubleData(result)
+  }
+
+  test("Test long vector datatype") {
+    prestoServer.execute("drop table if exists presto_spark_db.array_long")
+    prestoServer
+      .execute(
+        "create table presto_spark_db.array_long(salary array(bigint) ) with" +
+        "(format='CARBON') ")
+    copyStoreContents("array_long")
+    val result: List[Map[String, Any]] = prestoServer
+      .executeQuery("SELECT * FROM presto_spark_db.array_long")
+    assert(result.size == 3)
+    PrestoTestUtil.validateLongData(result)
+  }
+
+  test("Test timestamp vector datatype") {
+    prestoServer.execute("drop table if exists 
presto_spark_db.array_timestamp")
+    prestoServer
+      .execute(
+        "create table presto_spark_db.array_timestamp(time array<timestamp>) 
with" +
+        "(format='CARBON') ")
+    copyStoreContents("array_timestamp")
+    val result: List[Map[String, Any]] = prestoServer
+      .executeQuery("SELECT * FROM presto_spark_db.array_timestamp")
+    assert(result.size == 2)
+    PrestoTestUtil.validateTimestampData(result)
+  }
+
   test("Test streaming ") {
     prestoServer.execute("drop table if exists 
presto_spark_db.streaming_table")
     prestoServer
@@ -338,11 +402,7 @@ class PrestoTestUsingSparkStore
     val result: List[Map[String, Any]] = prestoServer
       .executeQuery("SELECT * FROM presto_spark_db.array_decimal")
     assert(result.size == 1)
-    val data = result(0)("salary")
-      .asInstanceOf[PrestoArray]
-      .getArray()
-      .asInstanceOf[Array[Object]]
-    assert(data.sameElements(Array("922.580", "3.435")))
+    PrestoTestUtil.validateDecimalData(result)
   }
 
   test("Test decimal unscaled converter for struct") {
diff --git 
a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
 
b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
index 3f3ae3c..56a9c48 100644
--- 
a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
+++ 
b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkStoreCreatorForPresto.scala
@@ -18,7 +18,7 @@
 package org.apache.carbondata.integration.spark.testsuite.dataload
 
 import java.io.{File, PrintWriter}
-import java.util.UUID
+import java.util.{TimeZone, UUID}
 
 import scala.util.Random
 
@@ -92,6 +92,11 @@ class SparkStoreCreatorForPresto extends QueryTest with 
BeforeAndAfterAll{
     sql("drop table if exists streaming_table")
     sql("drop table if exists array_decimal")
     sql("drop table if exists struct_decimal")
+    sql("drop table if exists array_short")
+    sql("drop table if exists array_int")
+    sql("drop table if exists array_long")
+    sql("drop table if exists array_double")
+    sql("drop table if exists array_timestamp")
     sql("use default ")
   }
 
@@ -385,6 +390,85 @@ class SparkStoreCreatorForPresto extends QueryTest with 
BeforeAndAfterAll{
     sql("insert into struct_decimal select named_struct('dec',922.580) ")
   }
 
+  test("Test short vector datatype") {
+    sql("drop table if exists array_short")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_short (salary array<short>) STORED AS 
" +
+      "carbondata"
+    )
+    sql("insert into array_short select array(4352,35,3) ") // page datatype - 
short, adaptive
+    // integral codec
+  }
+
+  test("Test int vector datatype") {
+    sql("drop table if exists array_int")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_int (salary array<int>) STORED AS " +
+      "carbondata"
+    )
+    sql("insert into array_int select array(21474836,21474839,23,3) ") // page 
datatype - int,
+    // adaptive integral codec
+
+    sql("insert into array_int select array(21474836,21474839) ") // page 
datatype - byte, adaptive
+    // delta integral codec
+
+  }
+
+  test("Test long vector datatype") {
+    sql("drop table if exists array_long")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_long (salary array<long>) STORED AS " +
+      "carbondata"
+    )
+    // following are for adaptive integral codec
+    sql("insert into array_long select array(215,23,3) ") // page datatype - 
short
+
+    sql("insert into array_long select array(32800,23,3) ") // page datatype - 
short_int
+
+    sql("insert into array_long select array(32800,214748364,3) ") // page 
datatype - int
+
+  }
+
+  test("Test double vector datatype") {
+    sql("drop table if exists array_double")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_double (salary array<double>) STORED 
AS " +
+      "carbondata"
+    )
+    // following are for adaptive integral codec
+    sql("insert into array_double select array(2,3,4) ") // page datatype - 
byte
+
+    sql("insert into array_double select array(242,35,43) ") // page datatype 
- short
+
+    sql("insert into array_double select array(32799,32767) ") // page 
datatype - short_int
+
+    sql("insert into array_double select array(21546546,32546546,43211564) ") 
// page datatype - int
+
+    // following are for adaptive floating codec
+    sql("insert into array_double select array(327.99,3.2799) ") // page 
datatype - short_int
+
+    sql("insert into array_double select array(1,2345,108787.123) ") // page 
datatype - int
+
+  }
+
+  test("Test timestamp vector datatype") {
+    val default_timezone = TimeZone.getDefault
+    TimeZone.setDefault(TimeZone.getTimeZone("Asia/Kolkata"))
+    sql("drop table if exists array_timestamp")
+    sql(
+      "CREATE TABLE IF NOT EXISTS array_timestamp (time array<timestamp>) 
STORED AS " +
+      "carbondata"
+    )
+    sql("insert into array_timestamp select array('2020-01-11 
12:00:45.0','2020-01-11 12:01:45.0')")
+    // page datatype - short-int, adaptive delta integral codec
+
+    sql("insert into array_timestamp select array('2020-01-10 
12:30:45.0','2015-01-11 12:01:45.0')")
+    // page datatype - long, adaptive integral
+
+    // set timezone back to default
+    TimeZone.setDefault(default_timezone)
+  }
+
   private def createFile(fileName: String, line: Int = 10000, start: Int = 0) 
= {
     if (!new File(fileName).exists()) {
       val write = new PrintWriter(new File(fileName))

Reply via email to