Repository: incubator-carbondata Updated Branches: refs/heads/master 7f8401d7f -> e882aedbe
[CARBONDATA-634] Load Query options invalid values are not consistent behavior. Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/80319548 Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/80319548 Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/80319548 Branch: refs/heads/master Commit: 80319548a2afb7a485a2401e117a3164f760da46 Parents: 7f8401d Author: mohammadshahidkhan <mohdshahidkhan1...@gmail.com> Authored: Wed Jan 18 12:01:23 2017 +0530 Committer: mohammadshahidkhan <mohdshahidkhan1...@gmail.com> Committed: Wed Jan 18 12:02:26 2017 +0530 ---------------------------------------------------------------------- .../badrecordloger/BadRecordLoggerTest.scala | 18 ++++++++++++++++++ .../spark/sql/catalyst/CarbonDDLSqlParser.scala | 13 +++++++++++++ 2 files changed, 31 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/80319548/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerTest.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerTest.scala index 8ef2fdd..005a6eb 100644 --- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerTest.scala +++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerTest.scala @@ -46,6 +46,7 @@ class BadRecordLoggerTest extends QueryTest with BeforeAndAfterAll { sql("drop table IF EXISTS emptyColumnValues_false") sql("drop table IF EXISTS empty_timestamp") sql("drop table IF EXISTS empty_timestamp_false") + sql("drop table IF EXISTS dataloadOptionTests") sql( """CREATE TABLE IF NOT EXISTS sales(ID BigInt, date Timestamp, country String, actual_price Double, Quantity int, sold_price Decimal(19,2)) STORED BY 'carbondata'""") @@ -237,6 +238,22 @@ class BadRecordLoggerTest extends QueryTest with BeforeAndAfterAll { ) } + test("test load ddl command") { + sql( + """CREATE TABLE IF NOT EXISTS dataloadOptionTests(ID BigInt, date Timestamp, country + String, + actual_price Double, Quantity int, sold_price Decimal(19,2)) STORED BY 'carbondata' + """) + val csvFilePath = s"$resourcesPath/badrecords/emptyTimeStampValue.csv" + try { + sql("LOAD DATA local inpath '" + csvFilePath + "' INTO TABLE dataloadOptionTests OPTIONS" + + "('bad_records_action'='FORCA', 'DELIMITER'= ',', 'QUOTECHAR'= '\"')"); + } catch { + case ex: Exception => + assert("option BAD_RECORDS_ACTION can have only either FORCE or IGNORE or REDIRECT" + .equals(ex.getMessage)) + } + } override def afterAll { sql("drop table sales") @@ -248,6 +265,7 @@ class BadRecordLoggerTest extends QueryTest with BeforeAndAfterAll { sql("drop table emptyColumnValues_false") sql("drop table empty_timestamp") sql("drop table empty_timestamp_false") + sql("drop table dataloadOptionTests") CarbonProperties.getInstance() .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy") } http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/80319548/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala index 166534f..742a145 100644 --- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala +++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala @@ -34,6 +34,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory import org.apache.carbondata.core.constants.CarbonCommonConstants import org.apache.carbondata.core.metadata.datatype.DataType import org.apache.carbondata.core.util.DataTypeUtil +import org.apache.carbondata.processing.constants.LoggerAction import org.apache.carbondata.spark.exception.MalformedCarbonCommandException import org.apache.carbondata.spark.util.CommonUtil @@ -782,6 +783,18 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser { } } + if (options.exists(_._1.equalsIgnoreCase("BAD_RECORDS_ACTION"))) { + val optionValue: String = options.get("bad_records_action").get.head._2 + try { + LoggerAction.valueOf(optionValue.toUpperCase) + } + catch { + case e: IllegalArgumentException => + throw new MalformedCarbonCommandException( + "option BAD_RECORDS_ACTION can have only either FORCE or IGNORE or REDIRECT") + } + } + // check for duplicate options val duplicateOptions = options filter { case (_, optionlist) => optionlist.size > 1