You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ku...@apache.org on 2020/10/27 09:55:43 UTC

[carbondata] branch master updated: [CARBONDATA-4007] Fix multiple issues in SDK_IUD

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

kunalkapoor 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 d68b7fa  [CARBONDATA-4007] Fix multiple issues in SDK_IUD
d68b7fa is described below

commit d68b7fa80e744640f9173e09c58d00caf5c019b3
Author: Karan980 <ka...@gmail.com>
AuthorDate: Wed Oct 7 12:16:33 2020 +0530

    [CARBONDATA-4007] Fix multiple issues in SDK_IUD
    
    Why is this PR needed?
    Fix multiple issues occurred in SDK_IUD.
    a) TupleId always have linux file separator independent of the system.
    b) Filtered rows array size gives ArrayOutOfBound exception if number of deleted rows is greater than 4096.
    c) On read, date columns were returned as integers and timestamp columns were returned as long.
    
    What changes were proposed in this PR?
    a) Changed the tupleId file separator to linux file separator.
    b) Change the filtered rows size to default column page rows size.
    c) Converted the date and timestamp columns to their respective forms on read.
    
    This closes #3970
---
 .../examples/sdk/CarbonReaderExample.java          |  8 +--
 .../carbondata/hadoop/AbstractRecordReader.java    |  3 ++
 .../carbondata/hadoop/CarbonRecordReader.java      |  5 ++
 .../hadoop/api/CarbonFileInputFormat.java          |  4 +-
 .../hadoop/api/CarbonTableOutputFormat.java        |  5 +-
 .../hadoop/util/CarbonVectorizedRecordReader.java  |  8 ++-
 .../carbondata/hive/CarbonHiveRecordReader.java    |  5 ++
 .../presto/PrestoCarbonVectorizedRecordReader.java |  5 ++
 .../vectorreader/VectorizedCarbonRecordReader.java |  5 ++
 .../org/apache/carbondata/sdk/file/CarbonIUD.java  |  1 -
 .../apache/carbondata/sdk/file/CarbonReader.java   | 57 ++++++++++++++++++++--
 .../carbondata/sdk/file/arrow/ArrowUtils.java      |  9 +---
 .../carbondata/sdk/file/ArrowCarbonReaderTest.java | 14 ++++++
 .../apache/carbondata/sdk/file/CarbonIUDTest.java  | 46 +++++++++++++++++
 .../carbondata/sdk/file/CarbonReaderTest.java      | 31 +++++-------
 15 files changed, 165 insertions(+), 41 deletions(-)

diff --git a/examples/spark/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java b/examples/spark/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
index eb09fe7..5ee342d 100644
--- a/examples/spark/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
+++ b/examples/spark/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
@@ -20,8 +20,6 @@ package org.apache.carbondata.examples.sdk;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
-import java.sql.Date;
-import java.sql.Timestamp;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -114,13 +112,11 @@ public class CarbonReaderExample {
           .build();
 
       System.out.println("\nData:");
-      long day = 24L * 3600 * 1000;
       int i = 0;
       while (reader.hasNext()) {
         Object[] row = (Object[]) reader.readNextRow();
         System.out.println(String.format("%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t",
-            i, row[0], row[1], row[2], row[3], row[4], row[5],
-            new Date((day * ((int) row[6]))), new Timestamp((long) row[7] / 1000),
+            i, row[0], row[1], row[2], row[3], row[4], row[5], row[6], row[7],
             row[8], row[9]
         ));
         Object[] arr = (Object[]) row[10];
@@ -144,7 +140,7 @@ public class CarbonReaderExample {
       while (reader2.hasNext()) {
         Object[] row = (Object[]) reader2.readNextRow();
         System.out.print(String.format("%s\t%s\t%s\t%s\t%s\t",
-            i, row[0], new Date((day * ((int) row[1]))), new Timestamp((long) row[2] / 1000),
+            i, row[0], row[1], row[2],
             row[3]));
         Object[] arr = (Object[]) row[4];
         for (int j = 0; j < arr.length; j++) {
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/AbstractRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/AbstractRecordReader.java
index 15898d3..e59a94d 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/AbstractRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/AbstractRecordReader.java
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.hadoop;
 
+import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
 import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
@@ -41,4 +42,6 @@ public abstract class AbstractRecordReader<T> extends RecordReader<Void, T> {
       recorder.recordStatistics(queryStatistic);
     }
   }
+
+  public abstract QueryModel getQueryModel();
 }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
index 2c5c821..3a3e599 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
@@ -118,6 +118,11 @@ public class CarbonRecordReader<T> extends AbstractRecordReader<T> {
   }
 
   @Override
+  public QueryModel getQueryModel() {
+    return queryModel;
+  }
+
+  @Override
   public boolean nextKeyValue() {
     return carbonIterator.hasNext();
   }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
index 7922244..2a655e8 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
@@ -279,7 +279,9 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
   private String[] getDeleteDeltaFiles(String segmentFilePath, List<String> allDeleteDeltaFiles) {
     List<String> deleteDeltaFiles = new ArrayList<>();
     String segmentFileName = null;
-    String[] pathElements = segmentFilePath.split(Pattern.quote(File.separator));
+    segmentFilePath = segmentFilePath.replace(CarbonCommonConstants.WINDOWS_FILE_SEPARATOR,
+            CarbonCommonConstants.FILE_SEPARATOR);
+    String[] pathElements = segmentFilePath.split(CarbonCommonConstants.FILE_SEPARATOR);
     if (ArrayUtils.isNotEmpty(pathElements)) {
       segmentFileName = pathElements[pathElements.length - 1];
     }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
index de8f05a..c7ba7a6 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.hadoop.api;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -29,7 +28,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.regex.Pattern;
 
 import org.apache.carbondata.common.exceptions.DeprecatedFeatureException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -578,7 +576,8 @@ public class CarbonTableOutputFormat extends FileOutputFormat<NullWritable, Obje
         String blockName;
         for (String tuple : tupleId) {
           blockName = CarbonUpdateUtil.getBlockName(
-              (tuple.split(Pattern.quote(File.separator))[TupleIdEnum.BLOCK_ID.getTupleIdIndex()]));
+              (tuple.split(CarbonCommonConstants.FILE_SEPARATOR)
+                      [TupleIdEnum.BLOCK_ID.getTupleIdIndex()]));
 
           if (!blockToDeleteDeltaBlockMapping.containsKey(blockName)) {
             blockDetails = new DeleteDeltaBlockDetails(blockName);
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonVectorizedRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonVectorizedRecordReader.java
index 2d6626a..cffae98 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonVectorizedRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonVectorizedRecordReader.java
@@ -58,7 +58,6 @@ public class CarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
 
   private static final Logger LOGGER =
       LogServiceFactory.getLogService(CarbonVectorizedRecordReader.class.getName());
-  private static final int DEFAULT_BATCH_SIZE = 4 * 1024;
 
   private CarbonColumnarBatch carbonColumnarBatch;
 
@@ -137,6 +136,11 @@ public class CarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
     return true;
   }
 
+  @Override
+  public QueryModel getQueryModel() {
+    return queryModel;
+  }
+
   private boolean nextBatch() {
     carbonColumnarBatch.reset();
     if (iterator.hasNext()) {
@@ -194,7 +198,7 @@ public class CarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
       }
       carbonColumnarBatch = new CarbonColumnarBatch(vectors,
           CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT,
-          new boolean[DEFAULT_BATCH_SIZE]);
+          new boolean[CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT]);
     }
   }
 
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java
index af08c33..d4baf97 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java
@@ -116,6 +116,11 @@ class CarbonHiveRecordReader extends CarbonRecordReader<ArrayWritable>
   }
 
   @Override
+  public QueryModel getQueryModel() {
+    return queryModel;
+  }
+
+  @Override
   public Void createKey() {
     return null;
   }
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
index 04e0de5..6a3c5ed 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
@@ -116,6 +116,11 @@ class PrestoCarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
   }
 
   @Override
+  public QueryModel getQueryModel() {
+    return queryModel;
+  }
+
+  @Override
   public void close() throws IOException {
     logStatistics(rowCount, queryModel.getStatisticsRecorder());
     if (columnarBatch != null) {
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index ea972ed..96d0542 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -164,6 +164,11 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
   }
 
   @Override
+  public QueryModel getQueryModel() {
+    return queryModel;
+  }
+
+  @Override
   public boolean nextKeyValue() {
     resultBatch();
 
diff --git a/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java b/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java
index 15d2202..ed526b9 100644
--- a/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java
+++ b/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java
@@ -173,7 +173,6 @@ public class CarbonIUD {
     RecordWriter<NullWritable, ObjectArrayWritable> deleteDeltaWriter =
         CarbonTableOutputFormat.getDeleteDeltaRecordWriter(path);
     ObjectArrayWritable writable = new ObjectArrayWritable();
-
     while (reader.hasNext()) {
       Object[] row = (Object[]) reader.readNextRow();
       writable.set(Arrays.copyOfRange(row, row.length - 1, row.length));
diff --git a/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java b/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
index 4e24a63..c3ebdc3 100644
--- a/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
+++ b/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
@@ -18,6 +18,8 @@
 package org.apache.carbondata.sdk.file;
 
 import java.io.IOException;
+import java.sql.Date;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
@@ -25,7 +27,14 @@ import java.util.UUID;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.DateDirectDictionaryGenerator;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.hadoop.AbstractRecordReader;
 import org.apache.carbondata.hadoop.CarbonRecordReader;
 import org.apache.carbondata.hadoop.util.CarbonVectorizedRecordReader;
 
@@ -109,7 +118,45 @@ public class CarbonReader<T> {
    */
   public T readNextRow() throws IOException, InterruptedException {
     validateReader();
-    return currentReader.getCurrentValue();
+    T row = currentReader.getCurrentValue();
+    // For CarbonRow date and timestamp are already in their correct form
+    if (row instanceof CarbonRow) {
+      return row;
+    }
+    return formatDateAndTimeStamp((Object []) row);
+  }
+
+  public T formatDateAndTimeStamp(Object[] row) {
+    List<ProjectionDimension> dimensions = ((AbstractRecordReader) currentReader)
+            .getQueryModel().getProjectionDimensions();
+    String carbonDateFormat = CarbonProperties.getInstance()
+            .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT);
+    if (carbonDateFormat == null) {
+      carbonDateFormat = CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT;
+    }
+    SimpleDateFormat dateFormat = new SimpleDateFormat(carbonDateFormat);
+    String carbonTimeStampFormat = CarbonProperties.getInstance()
+            .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT);
+    if (carbonTimeStampFormat  == null) {
+      carbonTimeStampFormat  = CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT;
+    }
+    SimpleDateFormat timeStampFormat = new SimpleDateFormat(carbonTimeStampFormat);
+    for (ProjectionDimension dimension : dimensions) {
+      ColumnSchema columnSchema = dimension.getDimension().getColumnSchema();
+      if (columnSchema == null) {
+        continue;
+      }
+      DataType dataType = columnSchema.getDataType();
+      if (dataType == DataTypes.DATE) {
+        row[dimension.getOrdinal()] = dateFormat
+                .format(new Date(DateDirectDictionaryGenerator.MILLIS_PER_DAY
+                        * (int)row[dimension.getOrdinal()]));
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        row[dimension.getOrdinal()] = timeStampFormat
+                .format(new Date((long)row[dimension.getOrdinal()] / 1000));
+      }
+    }
+    return (T)row;
   }
 
   /**
@@ -122,7 +169,11 @@ public class CarbonReader<T> {
       if (batchValue == null) {
         return null;
       } else {
-        return batchValue.toArray();
+        Object[] rows =  batchValue.toArray();
+        for (Object row : rows) {
+          row = formatDateAndTimeStamp((Object[]) row);
+        }
+        return rows;
       }
     } else if (currentReader instanceof CarbonVectorizedRecordReader) {
       int batch = Integer.parseInt(CarbonProperties.getInstance()
@@ -131,7 +182,7 @@ public class CarbonReader<T> {
       batchRows = new Object[batch];
       int sum = 0;
       for (int i = 0; i < batch; i++) {
-        batchRows[i] = currentReader.getCurrentValue();
+        batchRows[i] = formatDateAndTimeStamp((Object[]) currentReader.getCurrentValue());
         sum++;
         if (i != batch - 1) {
           if (!hasNext()) {
diff --git a/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/arrow/ArrowUtils.java b/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/arrow/ArrowUtils.java
index a3ca7c4..1f1cae7 100644
--- a/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/arrow/ArrowUtils.java
+++ b/sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/arrow/ArrowUtils.java
@@ -32,9 +32,7 @@ import org.apache.carbondata.core.metadata.datatype.StructType;
 import org.apache.carbondata.sdk.file.Schema;
 
 import org.apache.arrow.memory.RootAllocator;
-import org.apache.arrow.vector.types.DateUnit;
 import org.apache.arrow.vector.types.FloatingPointPrecision;
-import org.apache.arrow.vector.types.TimeUnit;
 import org.apache.arrow.vector.types.pojo.ArrowType;
 import org.apache.arrow.vector.types.pojo.FieldType;
 
@@ -43,7 +41,8 @@ public class ArrowUtils {
   public static final RootAllocator rootAllocator = new RootAllocator(Long.MAX_VALUE);
 
   public static ArrowType toArrowType(DataType carbonDataType, String timeZoneId) {
-    if (carbonDataType == DataTypes.STRING || carbonDataType == DataTypes.VARCHAR) {
+    if (carbonDataType == DataTypes.STRING || carbonDataType == DataTypes.VARCHAR
+            || carbonDataType == DataTypes.TIMESTAMP || carbonDataType == DataTypes.DATE) {
       return ArrowType.Utf8.INSTANCE;
     } else if (carbonDataType == DataTypes.BYTE) {
       return new ArrowType.Int(DataTypes.BYTE.getSizeInBytes() * 8, true);
@@ -63,10 +62,6 @@ public class ArrowUtils {
       // instance of check is for findbugs, instead of datatypes check
       DecimalType decimal = (DecimalType) carbonDataType;
       return new ArrowType.Decimal(decimal.getPrecision(), decimal.getScale());
-    } else if (carbonDataType == DataTypes.TIMESTAMP) {
-      return new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId);
-    } else if (carbonDataType == DataTypes.DATE) {
-      return new ArrowType.Date(DateUnit.DAY);
     } else if (carbonDataType == DataTypes.BINARY) {
       return ArrowType.Binary.INSTANCE;
     } else {
diff --git a/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/ArrowCarbonReaderTest.java b/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/ArrowCarbonReaderTest.java
index cd2e519..5b85f0f 100644
--- a/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/ArrowCarbonReaderTest.java
+++ b/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/ArrowCarbonReaderTest.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.sdk.file;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -35,6 +36,7 @@ import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.FieldVector;
 import org.apache.arrow.vector.Float4Vector;
 import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.VarCharVector;
 import org.apache.arrow.vector.VectorLoader;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
@@ -42,6 +44,8 @@ import org.apache.commons.io.FileUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import static org.junit.Assert.assertArrayEquals;
+
 public class ArrowCarbonReaderTest extends TestCase {
 
   @Test
@@ -116,6 +120,16 @@ public class ArrowCarbonReaderTest extends TestCase {
       for (int i = 0; i < vectorSchemaRoot.getRowCount(); i++) {
         assertEquals(((Float4Vector)fieldVectors.get(12)).get(i), (float) 1.23);
       }
+      // validate date column
+      for (int i = 0; i < vectorSchemaRoot.getRowCount(); i++) {
+        assertArrayEquals(((VarCharVector)fieldVectors.get(1)).get(i),
+                "2019-03-02".getBytes((StandardCharsets.UTF_8)));
+      }
+      // validate timestamp column
+      for (int i = 0; i < vectorSchemaRoot.getRowCount(); i++) {
+        assertArrayEquals(((VarCharVector)fieldVectors.get(2)).get(i),
+                "2019-02-12 03:03:34".getBytes((StandardCharsets.UTF_8)));
+      }
       arrowRecordBatch.close();
       vectorSchemaRoot.close();
       bufferAllocator.close();
diff --git a/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonIUDTest.java b/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonIUDTest.java
index ecb4249..c8cf8da 100644
--- a/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonIUDTest.java
+++ b/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonIUDTest.java
@@ -22,6 +22,7 @@ import java.io.FilenameFilter;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.datatype.Field;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
@@ -31,6 +32,7 @@ import org.apache.carbondata.core.scan.expression.logical.AndExpression;
 import org.apache.carbondata.core.scan.expression.logical.OrExpression;
 
 import org.apache.avro.generic.GenericData;
+import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.commons.io.FileUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -73,6 +75,50 @@ public class CarbonIUDTest {
   }
 
   @Test
+  public void testUpdateOnDateType() throws Exception {
+    String path = "./testWriteFiles";
+    CarbonProperties.getInstance()
+            .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                    CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+            .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+                    CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+    FileUtils.deleteDirectory(new File(path));
+    Field[] fields = new Field[3];
+    fields[0] = new Field("intField", DataTypes.INT);
+    fields[1] = new Field("dateField", DataTypes.DATE);
+    fields[2] = new Field("timeField", DataTypes.TIMESTAMP);
+    CarbonWriter writer = CarbonWriter.builder()
+            .outputPath(path)
+            .withCsvInput(new Schema(fields))
+            .writtenBy("IUDTest")
+            .build();
+    for (int i = 0; i < 10; i++) {
+      String[] row2 = new String[]{
+              String.valueOf(i % 10000),
+              "2019-03-02",
+              "2019-02-12 03:03:34",
+      };
+      writer.write(row2);
+    }
+    writer.close();
+    CarbonIUD.getInstance().update(path, "intField", "0", "intField", "20").commit();
+    CarbonReader reader =
+            CarbonReader.builder(path).projection(new String[] { "intField", "dateField", "timeField" })
+                    .build();
+    int i = 0;
+    while (reader.hasNext()) {
+      Object[] row = (Object[]) reader.readNextRow();
+      assert ((int) row[0] != 0);
+      assert (row[1].equals("2019-03-02"));
+      assert (row[2].equals("2019-02-12 03:03:34"));
+      i++;
+    }
+    Assert.assertEquals(i, 10);
+    reader.close();
+    FileUtils.deleteDirectory(new File(path));
+  }
+
+  @Test
   public void testDeleteWithConditionalExpressions() throws Exception {
     String path = "./testWriteFiles";
     FileUtils.deleteDirectory(new File(path));
diff --git a/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java b/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
index d47d5b3..33ac9a2 100644
--- a/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
+++ b/sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
@@ -50,8 +50,6 @@ import java.io.File;
 import java.io.FileFilter;
 import java.io.FilenameFilter;
 import java.io.IOException;
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
@@ -1148,9 +1146,8 @@ public class CarbonReaderTest extends TestCase {
       Assert.assertEquals(Long.MAX_VALUE - id, row[3]);
       Assert.assertEquals((double) id / 2, row[4]);
       Assert.assertEquals(true, (boolean) row[5]);
-      long day = 24L * 3600 * 1000;
-      Assert.assertEquals("2019-03-02", new Date((day * ((int) row[6]))).toString());
-      Assert.assertEquals("2019-02-12 03:03:34.0", new Timestamp((long) row[7] / 1000).toString());
+      Assert.assertEquals("2019-03-02", row[6]);
+      Assert.assertEquals("2019-02-12 03:03:34", row[7]);
       i++;
     }
     Assert.assertEquals(i, 100);
@@ -1259,9 +1256,8 @@ public class CarbonReaderTest extends TestCase {
       Assert.assertEquals(Long.MAX_VALUE - id, row[3]);
       Assert.assertEquals((double) id / 2, row[4]);
       Assert.assertEquals(true, (boolean) row[5]);
-      long day = 24L * 3600 * 1000;
-      Assert.assertEquals("2019-03-02", new Date((day * ((int) row[6]))).toString());
-      Assert.assertEquals("2019-02-12 03:03:34.0", new Timestamp((long) row[7] / 1000).toString());
+      Assert.assertEquals("2019-03-02", row[6]);
+      Assert.assertEquals("2019-02-12 03:03:34", row[7]);
       i++;
     }
     Assert.assertEquals(i, 100);
@@ -1371,9 +1367,8 @@ public class CarbonReaderTest extends TestCase {
       Assert.assertEquals(Long.MAX_VALUE - id, row[3]);
       Assert.assertEquals((double) id / 2, row[4]);
       Assert.assertEquals(true, (boolean) row[5]);
-      long day = 24L * 3600 * 1000;
-      Assert.assertEquals("2019-03-02", new Date((day * ((int) row[6]))).toString());
-      Assert.assertEquals("2019-02-12 03:03:34.0", new Timestamp((long) row[7] / 1000).toString());
+      Assert.assertEquals("2019-03-02", row[6]);
+      Assert.assertEquals("2019-02-12 03:03:34", row[7]);
       i++;
     }
     Assert.assertEquals(i, 100);
@@ -1732,7 +1727,7 @@ public class CarbonReaderTest extends TestCase {
         Object[] row = (Object[]) reader.readNextRow();
         assert (row[0].equals("robot" + i));
         assert (row[2].equals(i));
-        assert (row[6].equals(17957));
+        assert (row[6].equals("2019-03-02"));
         Object[] arr = (Object[]) row[10];
         assert (arr[0].equals("Hello"));
         assert (arr[3].equals("Carbon"));
@@ -1839,7 +1834,7 @@ public class CarbonReaderTest extends TestCase {
         assertEquals(RowUtil.getLong(data, 3), Long.MAX_VALUE - i);
         assertEquals(RowUtil.getDouble(data, 4), ((double) i) / 2);
         assert (RowUtil.getBoolean(data, 5));
-        assertEquals(RowUtil.getInt(data, 6), 17957);
+        assertEquals(RowUtil.getString(data, 6), "2019-03-02");
         assert (RowUtil.getDecimal(data, 8).equals("12.35"));
         assert (RowUtil.getVarchar(data, 9).equals("varchar"));
 
@@ -1924,7 +1919,7 @@ public class CarbonReaderTest extends TestCase {
         Object[] data = (Object[]) reader.readNextRow();
 
         assert (RowUtil.getString(data, 0).equals("robot" + i));
-        assertEquals(RowUtil.getInt(data, 1), 17957);
+        assertEquals(RowUtil.getString(data, 1), "2019-03-02");
         assert (RowUtil.getVarchar(data, 3).equals("varchar"));
         Object[] arr = RowUtil.getArray(data, 4);
         assert (arr[0].equals("Hello"));
@@ -2017,7 +2012,7 @@ public class CarbonReaderTest extends TestCase {
         assert (RowUtil.getLong(data, 6) == Long.MAX_VALUE - i);
         assertEquals(RowUtil.getDouble(data, 7), ((double) i) / 2);
         assert (RowUtil.getBoolean(data, 8));
-        assertEquals(RowUtil.getInt(data, 1), 17957);
+        assertEquals(RowUtil.getString(data, 1), "2019-03-02");
         assert (RowUtil.getDecimal(data, 9).equals("12.35"));
         assert (RowUtil.getString(data, 3).equals("varchar"));
         assertEquals(RowUtil.getByte(data, 10), new Byte(String.valueOf(i)));
@@ -2102,7 +2097,7 @@ public class CarbonReaderTest extends TestCase {
 
           Object[] data = (Object[]) batch[j];
           assert (RowUtil.getString(data, 0).equals("robot" + i));
-          assertEquals(RowUtil.getInt(data, 1), 17957);
+          assertEquals(RowUtil.getString(data, 1), "2019-03-02");
           assert (RowUtil.getVarchar(data, 3).equals("varchar"));
           Object[] arr = RowUtil.getArray(data, 4);
           assert (arr[0].equals("Hello"));
@@ -2196,7 +2191,7 @@ public class CarbonReaderTest extends TestCase {
 
           Object[] data = (Object[]) batch[j];
           assert (RowUtil.getString(data, 0).equals("robot" + i));
-          assertEquals(RowUtil.getInt(data, 1), 17957);
+          assertEquals(RowUtil.getString(data, 1), "2019-03-02");
           assert (RowUtil.getVarchar(data, 3).equals("varchar"));
           assertEquals(RowUtil.getShort(data, 4), i);
           assertEquals(RowUtil.getInt(data, 5), i);
@@ -2584,7 +2579,7 @@ public class CarbonReaderTest extends TestCase {
         Object[] data = (Object[]) reader.readNextRow();
 
         assert (RowUtil.getString(data, 0).equals("robot" + i));
-        assertEquals(RowUtil.getInt(data, 1), 17957);
+        assertEquals(RowUtil.getString(data, 1), "2019-03-02");
         Assert.assertEquals(new String(value), new String(RowUtil.getBinary(data, 3)));
         assert (RowUtil.getVarchar(data, 4).equals("varchar"));
         Object[] arr = RowUtil.getArray(data, 5);