This is an automated email from the ASF dual-hosted git repository.
akashrn5 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 984a32f [CARBONDATA-3964] Fixed, null pointer excption for select
query and time zone dependent test failures.
984a32f is described below
commit 984a32fd0517225dde267393993078cc683ee196
Author: Nihal ojha <[email protected]>
AuthorDate: Thu Aug 27 13:58:29 2020 +0530
[CARBONDATA-3964] Fixed, null pointer excption for select query and time
zone dependent test
failures.
Why is this PR needed?
1. In case of 1 million record and 500 segments select query without filter
is thowing null
pointer exception.
2. Currently some testcases are failing releated to timestamp and time zone.
What changes were proposed in this PR?
1. Select query without filter should execute pruneWithoutFilter method
rather than
pruneWithMultiThread. Added null check for filter.
2. According to timezone version releases for Java we have different DST
rule for different
JRE version. In this PR we have Changed the testcase timezone and date to
be compatible.
Does this PR introduce any user interface change?
No
Is any new testcase added?
No
This closes #3905
---
.../apache/carbondata/core/index/TableIndex.java | 4 +-
.../TestLoadDataWithDiffTimestampFormat.scala | 55 ++++++++++++++--------
.../carbondata/spark/util/BadRecordUtil.scala | 2 +-
3 files changed, 40 insertions(+), 21 deletions(-)
diff --git
a/core/src/main/java/org/apache/carbondata/core/index/TableIndex.java
b/core/src/main/java/org/apache/carbondata/core/index/TableIndex.java
index c4c600a..fe5e756 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/TableIndex.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/TableIndex.java
@@ -152,8 +152,10 @@ public final class TableIndex extends
OperationEventListener {
int numOfThreadsForPruning = CarbonProperties.getNumOfThreadsForPruning();
int carbonDriverPruningMultiThreadEnableFilesCount =
CarbonProperties.getDriverPruningMultiThreadEnableFilesCount();
+ // when the query is without filter, as we need to return all the
blocklets,
+ // so no need of multi-thread pruning
if (numOfThreadsForPruning == 1 || indexesCount < numOfThreadsForPruning
|| totalFiles
- < carbonDriverPruningMultiThreadEnableFilesCount) {
+ < carbonDriverPruningMultiThreadEnableFilesCount ||
!isFilterPresent) {
// use multi-thread, only if the files are more than 0.1 million.
// As 0.1 million files block pruning can take only 1 second.
// Doing multi-thread for smaller values is not recommended as
diff --git
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
index 319ad4e..63a2fc6 100644
---
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
+++
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
@@ -318,16 +318,24 @@ class TestLoadDataWithDiffTimestampFormat extends
QueryTest with BeforeAndAfterA
test("test load, update data with setlenient carbon property for daylight " +
"saving time from different timezone") {
CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_LOAD_DATEFORMAT_SETLENIENT_ENABLE,
"true")
- TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
sql("DROP TABLE IF EXISTS test_time")
+ sql("DROP TABLE IF EXISTS testhivetable")
+ // Create test_time and hive table
sql("CREATE TABLE IF NOT EXISTS test_time (ID Int, date Date, time
Timestamp) STORED AS carbondata " +
"TBLPROPERTIES('dateformat'='yyyy-MM-dd',
'timestampformat'='yyyy-MM-dd HH:mm:ss') ")
- sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv'
into table test_time")
- sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
- sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
- checkAnswer(sql("SELECT time FROM test_time WHERE ID = 1"),
Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
- checkAnswer(sql("SELECT time FROM test_time WHERE ID = 11"),
Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
- checkAnswer(sql("SELECT time FROM test_time WHERE ID = 2"),
Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+ sql("CREATE TABLE testhivetable (ID Int, date Date, time TIMESTAMP) row
format delimited fields terminated by ',' ")
+ // load data into test_time and hive table and validate query result
+ sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv'
into table test_time options('fileheader'='ID,date,time')")
+ sql(s"LOAD DATA local inpath '$resourcesPath/differentZoneTimeStamp.csv'
overwrite INTO table testhivetable")
+ checkAnswer(sql("select * from test_time"), sql("select * from
testhivetable"))
+ sql(s"insert into test_time select 11, '2016-7-24', '2019-3-10 02:00:00' ")
+ sql("update test_time set (time) = ('2019-3-10 02:00:00') where ID='2'")
+ // Using America/Los_Angeles timezone (timezone is fixed to
America/Los_Angeles for all tests)
+ // Here, 2019-3-10 02:00:00 is invalid data in America/Los_Angeles zone,
as DST is observed and
+ // clocks were turned forward 1 hour to 2019-3-10 03:00:00. With lenience
property enabled, can parse the time according to DST.
+ checkAnswer(sql("SELECT time FROM test_time WHERE ID = 1"),
Seq(Row(Timestamp.valueOf("2019-3-10 03:00:00"))))
+ checkAnswer(sql("SELECT time FROM test_time WHERE ID = 11"),
Seq(Row(Timestamp.valueOf("2019-3-10 03:00:00"))))
+ checkAnswer(sql("SELECT time FROM test_time WHERE ID = 2"),
Seq(Row(Timestamp.valueOf("2019-3-10 03:00:00"))))
sql("DROP TABLE test_time")
CarbonProperties.getInstance().removeProperty(CarbonCommonConstants.CARBON_LOAD_DATEFORMAT_SETLENIENT_ENABLE)
}
@@ -335,24 +343,32 @@ class TestLoadDataWithDiffTimestampFormat extends
QueryTest with BeforeAndAfterA
test("test load, update data with setlenient session level property for
daylight " +
"saving time from different timezone") {
sql("set carbon.load.dateformat.setlenient.enable = true")
- TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
sql("DROP TABLE IF EXISTS test_time")
- sql("CREATE TABLE IF NOT EXISTS test_time (ID Int, date Date, time
Timestamp) STORED AS carbondata " +
+ sql("DROP TABLE IF EXISTS testhivetable")
+ // Create test_time and hive table
+ sql("CREATE TABLE test_time (ID Int, date Date, time Timestamp) STORED AS
carbondata " +
"TBLPROPERTIES('dateformat'='yyyy-MM-dd',
'timestampformat'='yyyy-MM-dd HH:mm:ss') ")
- sql(s" LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv'
into table test_time")
- sql(s"insert into test_time select 11, '2016-7-24', '1941-3-15 00:00:00' ")
- sql("update test_time set (time) = ('1941-3-15 00:00:00') where ID='2'")
- checkAnswer(sql("SELECT time FROM test_time WHERE ID = 1"),
Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
- checkAnswer(sql("SELECT time FROM test_time WHERE ID = 11"),
Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
- checkAnswer(sql("SELECT time FROM test_time WHERE ID = 2"),
Seq(Row(Timestamp.valueOf("1941-3-15 01:00:00"))))
+ sql("CREATE TABLE testhivetable (ID Int, date Date, time TIMESTAMP) row
format delimited fields terminated by ',' ")
+ // load data into test_time and hive table and validate query result
+ sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/differentZoneTimeStamp.csv'
into table test_time options('fileheader'='ID,date,time')")
+ sql(s"LOAD DATA local inpath '$resourcesPath/differentZoneTimeStamp.csv'
overwrite INTO table testhivetable")
+ checkAnswer(sql("select * from test_time"), sql("select * from
testhivetable"))
+ sql(s"insert into test_time select 11, '2016-7-24', '2019-3-10 02:00:00' ")
+ sql("update test_time set (time) = ('2019-3-10 02:00:00') where ID='2'")
+ // Using America/Los_Angeles timezone (timezone is fixed to
America/Los_Angeles for all tests)
+ // Here, 2019-3-10 02:00:00 is invalid data in America/Los_Angeles zone,
as DST is observed and
+ // clocks were turned forward 1 hour to 2019-3-10 03:00:00. With lenience
property enabled, can parse the time according to DST.
+ checkAnswer(sql("SELECT time FROM test_time WHERE ID = 1"),
Seq(Row(Timestamp.valueOf("2019-3-10 03:00:00"))))
+ checkAnswer(sql("SELECT time FROM test_time WHERE ID = 11"),
Seq(Row(Timestamp.valueOf("2019-3-10 03:00:00"))))
+ checkAnswer(sql("SELECT time FROM test_time WHERE ID = 2"),
Seq(Row(Timestamp.valueOf("2019-3-10 03:00:00"))))
+ sql("DROP TABLE testhivetable")
sql("DROP TABLE test_time")
- defaultConfig()
+ sql("set carbon.load.dateformat.setlenient.enable = false")
}
def generateCSVFile(): Unit = {
val rows = new ListBuffer[Array[String]]
- rows += Array("ID", "date", "time")
- rows += Array("1", "1941-3-15", "1941-3-15 00:00:00")
+ rows += Array("1", "1941-3-15", "2019-3-10 02:00:00")
rows += Array("2", "2016-7-24", "2016-7-24 01:02:30")
BadRecordUtil.createCSV(rows, csvPath)
}
@@ -360,6 +376,7 @@ class TestLoadDataWithDiffTimestampFormat extends QueryTest
with BeforeAndAfterA
override def afterAll {
sql("DROP TABLE IF EXISTS t3")
FileUtils.forceDelete(new File(csvPath))
- TimeZone.setDefault(defaultTimeZone)
+
CarbonProperties.getInstance().removeProperty(CarbonCommonConstants.CARBON_LOAD_DATEFORMAT_SETLENIENT_ENABLE)
+ sql("set carbon.load.dateformat.setlenient.enable = false")
}
}
diff --git
a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
index 3f4fa60..4eb7c82 100644
---
a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
+++
b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
@@ -73,7 +73,7 @@ object BadRecordUtil {
def createCSV(rows: ListBuffer[Array[String]], csvPath: String): Unit = {
val out = new BufferedWriter(new FileWriter(csvPath))
- val writer: CSVWriter = new CSVWriter(out)
+ val writer: CSVWriter = new CSVWriter(out, ',',
CSVWriter.NO_QUOTE_CHARACTER)
try {
for (row <- rows) {
writer.writeNext(row)