[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-21 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r474543420



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -145,47 +152,153 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("drop table if exists carbon_table")
   }
 
-  test("test insert / update with data more than 32000 characters") {
+  private def createTableAndLoadData (badRecordAction: String): Unit = {
+BadRecordUtil.cleanBadRecordPath("default", "longerthan32kchar")
+sql("CREATE TABLE longerthan32kchar(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
+sql(s"LOAD DATA LOCAL INPATH '$testdata' into table longerThan32kChar 
OPTIONS('FILEHEADER'='dim1,dim2,mes1', " +
+  
s"'BAD_RECORDS_ACTION'='${badRecordAction}','BAD_RECORDS_LOGGER_ENABLE'='TRUE')")
+  }
+
+  test("test load / insert / update with data more than 32000 characters and 
bad record action as Redirect") {
+createTableAndLoadData("REDIRECT")
+var redirectCsvPath = BadRecordUtil
+  .getRedirectCsvPath("default", "longerthan32kchar", "0", "0")
+assert(BadRecordUtil.checkRedirectedCsvContentAvailableInSource(testdata, 
redirectCsvPath))
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "true")
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, 
"REDIRECT");
+sql(s"insert into longerthan32kchar values('33000', '$longChar', 4)")
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("ok", "hi", 
1), Row("itsok", "hello", 2)))
+redirectCsvPath = BadRecordUtil.getRedirectCsvPath("default", 
"longerthan32kchar", "1", "0")
+var redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+var iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("33000,"+longChar+",4"))
+}
+
+// Update strings of length greater than 32000
+sql(s"update longerthan32kchar set(longerthan32kchar.dim2)=('$longChar') " 
+
+  "where longerthan32kchar.mes1=1").show()
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("itsok", 
"hello", 2)))
+redirectCsvPath = BadRecordUtil.getRedirectCsvPath("default", 
"longerthan32kchar", "0", "1")
+redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("ok,"+longChar+",1"))
+}
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "false")
+
+// Insert longer string without converter step will throw exception
+intercept[Exception] {
+  sql(s"insert into longerthan32kchar values('32000', '$longChar', 3)")
+}
+BadRecordUtil.cleanBadRecordPath("default", "longerthan32kchar")
+  }
+
+  test("test load / insert / update with data more than 32000 characters and 
bad record action as Force") {
+createTableAndLoadData("FORCE")
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("ok", "hi", 
1), Row("itsok", "hello", 2), Row("32123", null, 3)))
 CarbonProperties.getInstance()
   
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "true")
-val testdata =s"$resourcesPath/32000char.csv"
-sql("drop table if exists load32000chardata")
-sql("drop table if exists load32000chardata_dup")
-sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
-sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
-sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FORCE");
+sql(s"insert into longerthan32kchar values('33000', '$longChar', 4)")
+checkAnswer(sql("select * from longerthan32kchar"),
+  Seq(Row("ok", "hi", 1), Row("itsok", "hello", 2), Row("32123", null, 3), 
Row("33000", null, 4)))
+
+// Update strings of length greater than 32000
+sql(s"update longerthan32kchar set(longerthan32kchar.dim2)=('$longChar') " 
+
+  "where longerthan32kchar.mes1=1").show()
+checkAnswer(sql("select * from longerthan32kchar"),
+  Seq(Row("ok", null, 1), Row("itsok", "hello", 2), Row("32123", null, 3), 
Row("33000", null, 4)))
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "false")
+
+// Insert longer string without converter step will throw exception
 intercept[Exception] {
-  sql("insert into load32000chardata_dup select 

[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-21 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r474540605



##
File path: 
streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala
##
@@ -54,11 +54,12 @@ object FieldConverter {
   value match {
 case s: String => if (!isVarcharType && !isComplexType &&
   s.length > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-  throw new IllegalArgumentException(stringLengthExceedErrorMsg +
-CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " characters")
-} else {
-  s
+  if (!CarbonProperties.getInstance.getProperty(CarbonCommonConstants

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-21 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r474518906



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
##
@@ -130,9 +135,14 @@ public CarbonRow convert(CarbonRow row) throws 
CarbonDataLoadingException {
   }
   fieldConverters[i].convert(row, logHolder);
   if (!logHolder.isLogged() && logHolder.isBadRecordNotAdded()) {
-badRecordLogger.addBadRecordsToBuilder(row.getRawData(), 
logHolder.getReason());
+String reason = logHolder.getReason();

Review comment:
   We have to set the record also in log and we don't pass the rawdata to 
`NonDictionaryFieldConverterImpl.convert()` . So we have to do formatting in 
`RowConverterImpl`





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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-20 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r474419263



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -145,47 +152,153 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("drop table if exists carbon_table")
   }
 
-  test("test insert / update with data more than 32000 characters") {
+  private def createTableAndLoadData (badRecordAction: String): Unit = {
+BadRecordUtil.cleanBadRecordPath("default", "longerthan32kchar")
+sql("CREATE TABLE longerthan32kchar(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
+sql(s"LOAD DATA LOCAL INPATH '$testdata' into table longerThan32kChar 
OPTIONS('FILEHEADER'='dim1,dim2,mes1', " +
+  
s"'BAD_RECORDS_ACTION'='${badRecordAction}','BAD_RECORDS_LOGGER_ENABLE'='TRUE')")
+  }
+
+  test("test load / insert / update with data more than 32000 characters and 
bad record action as Redirect") {
+createTableAndLoadData("REDIRECT")
+var redirectCsvPath = BadRecordUtil
+  .getRedirectCsvPath("default", "longerthan32kchar", "0", "0")
+assert(BadRecordUtil.checkRedirectedCsvContentAvailableInSource(testdata, 
redirectCsvPath))
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "true")
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, 
"REDIRECT");
+sql(s"insert into longerthan32kchar values('33000', '$longChar', 4)")
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("ok", "hi", 
1), Row("itsok", "hello", 2)))
+redirectCsvPath = BadRecordUtil.getRedirectCsvPath("default", 
"longerthan32kchar", "1", "0")
+var redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+var iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("33000,"+longChar+",4"))
+}
+
+// Update strings of length greater than 32000
+sql(s"update longerthan32kchar set(longerthan32kchar.dim2)=('$longChar') " 
+
+  "where longerthan32kchar.mes1=1").show()
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("itsok", 
"hello", 2)))
+redirectCsvPath = BadRecordUtil.getRedirectCsvPath("default", 
"longerthan32kchar", "0", "1")
+redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("ok,"+longChar+",1"))
+}
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "false")
+
+// Insert longer string without converter step will throw exception
+intercept[Exception] {

Review comment:
   we can't check here because this is system generated exception. Not user 
formated exception.





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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-20 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r474138074



##
File path: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##
@@ -2468,4 +2468,10 @@ private CarbonCommonConstants() {
* index server temp folder aging period default value 3hours.
*/
   public static final String CARBON_INDEXSERVER_TEMPFOLDER_DELETETIME_DEFAULT 
= "1080";
+
+  public static final String STRING_LENGTH_EXCEEDED_MESSAGE =
+  "Record %s of column %s exceeded " + MAX_CHARS_PER_COLUMN_DEFAULT +
+  " characters. Please consider long string data type.";
+
+  public static final String FORCE_BAD_RECORD_ACTION = "FORCE";

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-18 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r472300831



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##
@@ -0,0 +1,82 @@
+/*
+ * 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.spark.util
+
+import java.io.{File, FileFilter}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.commons.io.FileUtils
+
+object BadRecordUtil {
+
+  /**
+   * get the bad record redirected csv file path
+   * @param dbName

Review comment:
   done

##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##
@@ -0,0 +1,82 @@
+/*
+ * 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.spark.util
+
+import java.io.{File, FileFilter}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.commons.io.FileUtils
+
+object BadRecordUtil {
+
+  /**
+   * get the bad record redirected csv file path
+   * @param dbName
+   * @param tableName
+   * @param segment
+   * @param task
+   * @return csv File
+   */
+  def getRedirectCsvPath(dbName: String,
+tableName: String, segment: String, task: String): File = {
+var badRecordLocation = CarbonProperties.getInstance()
+  .getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC)
+badRecordLocation = badRecordLocation + "/" + dbName + "/" + tableName + 
"/" + segment + "/" +
+  task
+val listFiles = new File(badRecordLocation).listFiles(new FileFilter {
+  override def accept(pathname: File): Boolean = {
+pathname.getPath.endsWith(".csv")
+  }
+})
+listFiles(0)
+  }
+
+  /**
+   * compare data of csvfile and redirected csv file.
+   * @param csvFilePath csv file path

Review comment:
   done

##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/spark/util/BadRecordUtil.scala
##
@@ -0,0 +1,82 @@
+/*
+ * 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.spark.util
+
+import java.io.{File, FileFilter}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.commons.io.FileUtils
+
+object BadRecordUtil {
+
+  /**
+   * get the 

[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-18 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r472122694



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -145,47 +150,162 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("drop table if exists carbon_table")
   }
 
-  test("test insert / update with data more than 32000 characters") {
+  test("test load / insert / update with data more than 32000 characters and 
bad record action as Redirect") {
+val testdata =s"$resourcesPath/MoreThan32KChar.csv"
+FileFactory.deleteAllFilesOfDir(new File(CarbonProperties.getInstance()
+  .getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC)))
+sql("CREATE TABLE longerthan32kchar(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
+sql(s"LOAD DATA LOCAL INPATH '$testdata' into table longerThan32kChar 
OPTIONS('FILEHEADER'='dim1,dim2,mes1', " +
+  s"'BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE')")
+var redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", 
"0", "0")
+assert(checkRedirectedCsvContentAvailableInSource(testdata, 
redirectCsvPath))
+val longChar: String = RandomStringUtils.randomAlphabetic(33000)
+
 CarbonProperties.getInstance()
   
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "true")
-val testdata =s"$resourcesPath/32000char.csv"
-sql("drop table if exists load32000chardata")
-sql("drop table if exists load32000chardata_dup")
-sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
-sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
-sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, 
"REDIRECT");
+sql(s"insert into longerthan32kchar values('33000', '$longChar', 4)")
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("ok", "hi", 
1), Row("itsok", "hello", 2)))
+redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", "1", 
"0")
+var redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+var iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("33000,"+longChar+",4"))
+}
+
+// Update strings of length greater than 32000
+sql(s"update longerthan32kchar set(longerthan32kchar.dim2)=('$longChar') " 
+
+  "where longerthan32kchar.mes1=1").show()
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("itsok", 
"hello", 2)))
+redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", "0", 
"1")
+redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("ok,"+longChar+",1"))
+}
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "false")
+
+// Insert longer string without converter step will throw exception
 intercept[Exception] {

Review comment:
   Added at some place but here exception message is not user formatted





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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-18 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r472122959



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -145,47 +150,162 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("drop table if exists carbon_table")
   }
 
-  test("test insert / update with data more than 32000 characters") {
+  test("test load / insert / update with data more than 32000 characters and 
bad record action as Redirect") {
+val testdata =s"$resourcesPath/MoreThan32KChar.csv"
+FileFactory.deleteAllFilesOfDir(new File(CarbonProperties.getInstance()
+  .getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC)))
+sql("CREATE TABLE longerthan32kchar(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
+sql(s"LOAD DATA LOCAL INPATH '$testdata' into table longerThan32kChar 
OPTIONS('FILEHEADER'='dim1,dim2,mes1', " +
+  s"'BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE')")
+var redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", 
"0", "0")
+assert(checkRedirectedCsvContentAvailableInSource(testdata, 
redirectCsvPath))
+val longChar: String = RandomStringUtils.randomAlphabetic(33000)
+
 CarbonProperties.getInstance()
   
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "true")
-val testdata =s"$resourcesPath/32000char.csv"
-sql("drop table if exists load32000chardata")
-sql("drop table if exists load32000chardata_dup")
-sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
-sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
-sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, 
"REDIRECT");
+sql(s"insert into longerthan32kchar values('33000', '$longChar', 4)")
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("ok", "hi", 
1), Row("itsok", "hello", 2)))
+redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", "1", 
"0")
+var redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+var iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("33000,"+longChar+",4"))
+}
+
+// Update strings of length greater than 32000
+sql(s"update longerthan32kchar set(longerthan32kchar.dim2)=('$longChar') " 
+
+  "where longerthan32kchar.mes1=1").show()
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("itsok", 
"hello", 2)))
+redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", "0", 
"1")
+redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("ok,"+longChar+",1"))
+}
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "false")
+
+// Insert longer string without converter step will throw exception
 intercept[Exception] {
-  sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
+  sql(s"insert into longerthan32kchar values('32000', '$longChar', 3)")
 }
-sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata_dup 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
+
+FileFactory.deleteAllFilesOfDir(new File(CarbonProperties.getInstance()
+  .getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC)))
+  }
+
+  test("test load / insert / update with data more than 32000 characters and 
bad record action as Force") {
+val testdata =s"$resourcesPath/MoreThan32KChar.csv"
+sql("CREATE TABLE longerthan32kchar(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
+sql(s"LOAD DATA LOCAL INPATH '$testdata' into table longerThan32kChar 
OPTIONS('FILEHEADER'='dim1,dim2,mes1', " +
+  s"'BAD_RECORDS_ACTION'='FORCE','BAD_RECORDS_LOGGER_ENABLE'='TRUE')")
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("ok", "hi", 
1), Row("itsok", "hello", 2), Row("32123", null, 3)))
+val longChar: String = RandomStringUtils.randomAlphabetic(33000)
+
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "true")
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FORCE");
+sql(s"insert into longerthan32kchar values('33000', '$longChar', 4)")
+checkAnswer(sql("select * from longerthan32kchar"),
+  Seq(Row("ok", "hi", 1), Row("itsok", "hello", 2), 

[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-18 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r472122318



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
##
@@ -207,6 +208,45 @@ class QueryTest extends PlanTest {
   }
 }
   }
+
+  def getRedirectCsvPath(dbName: String,

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-18 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r472122106



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -145,47 +150,162 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("drop table if exists carbon_table")
   }
 
-  test("test insert / update with data more than 32000 characters") {
+  test("test load / insert / update with data more than 32000 characters and 
bad record action as Redirect") {
+val testdata =s"$resourcesPath/MoreThan32KChar.csv"
+FileFactory.deleteAllFilesOfDir(new File(CarbonProperties.getInstance()
+  .getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC)))
+sql("CREATE TABLE longerthan32kchar(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
+sql(s"LOAD DATA LOCAL INPATH '$testdata' into table longerThan32kChar 
OPTIONS('FILEHEADER'='dim1,dim2,mes1', " +
+  s"'BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE')")
+var redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", 
"0", "0")
+assert(checkRedirectedCsvContentAvailableInSource(testdata, 
redirectCsvPath))
+val longChar: String = RandomStringUtils.randomAlphabetic(33000)
+
 CarbonProperties.getInstance()
   
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "true")
-val testdata =s"$resourcesPath/32000char.csv"
-sql("drop table if exists load32000chardata")
-sql("drop table if exists load32000chardata_dup")
-sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
-sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
-sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, 
"REDIRECT");
+sql(s"insert into longerthan32kchar values('33000', '$longChar', 4)")
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("ok", "hi", 
1), Row("itsok", "hello", 2)))
+redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", "1", 
"0")
+var redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+var iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("33000,"+longChar+",4"))
+}
+
+// Update strings of length greater than 32000
+sql(s"update longerthan32kchar set(longerthan32kchar.dim2)=('$longChar') " 
+
+  "where longerthan32kchar.mes1=1").show()
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("itsok", 
"hello", 2)))
+redirectCsvPath = getRedirectCsvPath("default", "longerthan32kchar", "0", 
"1")
+redirectedFileLineList = FileUtils.readLines(redirectCsvPath)
+iterator = redirectedFileLineList.iterator()
+while (iterator.hasNext) {
+  assert(iterator.next().equals("ok,"+longChar+",1"))
+}
+CarbonProperties.getInstance()
+  
.addProperty(CarbonCommonConstants.CARBON_ENABLE_BAD_RECORD_HANDLING_FOR_INSERT,
 "false")
+
+// Insert longer string without converter step will throw exception
 intercept[Exception] {
-  sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
+  sql(s"insert into longerthan32kchar values('32000', '$longChar', 3)")
 }
-sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata_dup 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
+
+FileFactory.deleteAllFilesOfDir(new File(CarbonProperties.getInstance()
+  .getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC)))
+  }
+
+  test("test load / insert / update with data more than 32000 characters and 
bad record action as Force") {
+val testdata =s"$resourcesPath/MoreThan32KChar.csv"
+sql("CREATE TABLE longerthan32kchar(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
+sql(s"LOAD DATA LOCAL INPATH '$testdata' into table longerThan32kChar 
OPTIONS('FILEHEADER'='dim1,dim2,mes1', " +
+  s"'BAD_RECORDS_ACTION'='FORCE','BAD_RECORDS_LOGGER_ENABLE'='TRUE')")
+checkAnswer(sql("select * from longerthan32kchar"), Seq(Row("ok", "hi", 
1), Row("itsok", "hello", 2), Row("32123", null, 3)))

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471232261



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
##
@@ -82,21 +83,25 @@ public Object convert(Object value, BadRecordLogHolder 
logHolder)
   .getBytesBasedOnDataTypeForNoDictionaryColumn(dimensionValue, 
dataType, dateFormat);
   if (dataType == DataTypes.STRING
   && parsedValue.length > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException(String.format(
-"Dataload failed, String size cannot exceed %d bytes,"
-+ " please consider long string data type",
-CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT));
+
logHolder.setReason(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE);
+String badRecordAction = CarbonProperties.getInstance()
+.getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION);
+if 
(badRecordAction.equalsIgnoreCase(CarbonCommonConstants.FORCE_BAD_RECORD_ACTION))
 {
+  parsedValue = getNullValue();
+}
   }
   return parsedValue;
 } else {
   Object parsedValue = DataTypeUtil
   .getDataDataTypeForNoDictionaryColumn(dimensionValue, dataType, 
dateFormat);
   if (dataType == DataTypes.STRING && parsedValue.toString().length()
   > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException(String.format(
-"Dataload failed, String size cannot exceed %d bytes,"
-+ " please consider long string data type",
-CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT));
+
logHolder.setReason(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE);
+if (CarbonProperties.getInstance()
+.getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION)

Review comment:
   done and added test cases for all the actions.





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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471232072



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
##
@@ -82,21 +83,25 @@ public Object convert(Object value, BadRecordLogHolder 
logHolder)
   .getBytesBasedOnDataTypeForNoDictionaryColumn(dimensionValue, 
dataType, dateFormat);
   if (dataType == DataTypes.STRING
   && parsedValue.length > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException(String.format(
-"Dataload failed, String size cannot exceed %d bytes,"
-+ " please consider long string data type",
-CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT));
+
logHolder.setReason(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE);
+String badRecordAction = CarbonProperties.getInstance()

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471232002



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
##
@@ -330,21 +331,35 @@ public void writeByteArray(Object input, DataOutputStream 
dataOutputStream,
 }
   }
 
+  private byte[] getNullForBytes(byte[] value) {
+String badRecordAction = CarbonProperties.getInstance()
+.getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION);
+if 
(badRecordAction.equalsIgnoreCase(CarbonCommonConstants.FORCE_BAD_RECORD_ACTION))
 {
+  if (this.carbonDimension.getDataType() == DataTypes.STRING) {
+return CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+  } else {
+return CarbonCommonConstants.EMPTY_BYTE_ARRAY;
+  }
+}
+return value;
+  }
+
   private void checkAndWriteByteArray(Object input, DataOutputStream 
dataOutputStream,
   BadRecordLogHolder logHolder, Boolean isWithoutConverter, String 
parsedValue, byte[] value)
   throws IOException {
 if (isWithoutConverter) {
   if (this.carbonDimension.getDataType() == DataTypes.STRING && input 
instanceof String
   && ((String)input).length() > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException("Dataload failed, String size 
cannot exceed "
-+ CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes");
+
logHolder.setReason(String.format(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE,
+input.toString(), this.carbonDimension.getColName()));
+value = getNullForBytes(value);
   }
   updateValueToByteStream(dataOutputStream, value);
 } else {
   if (this.carbonDimension.getDataType() == DataTypes.STRING
   && value.length > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException("Dataload failed, String size 
cannot exceed "
-+ CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes");
+
logHolder.setReason(String.format(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE,

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471231678



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
##
@@ -330,21 +331,35 @@ public void writeByteArray(Object input, DataOutputStream 
dataOutputStream,
 }
   }
 
+  private byte[] getNullForBytes(byte[] value) {
+String badRecordAction = CarbonProperties.getInstance()
+.getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION);
+if 
(badRecordAction.equalsIgnoreCase(CarbonCommonConstants.FORCE_BAD_RECORD_ACTION))
 {
+  if (this.carbonDimension.getDataType() == DataTypes.STRING) {
+return CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+  } else {
+return CarbonCommonConstants.EMPTY_BYTE_ARRAY;
+  }
+}
+return value;
+  }
+
   private void checkAndWriteByteArray(Object input, DataOutputStream 
dataOutputStream,
   BadRecordLogHolder logHolder, Boolean isWithoutConverter, String 
parsedValue, byte[] value)
   throws IOException {
 if (isWithoutConverter) {
   if (this.carbonDimension.getDataType() == DataTypes.STRING && input 
instanceof String
   && ((String)input).length() > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException("Dataload failed, String size 
cannot exceed "
-+ CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes");
+
logHolder.setReason(String.format(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE,
+input.toString(), this.carbonDimension.getColName()));
+value = getNullForBytes(value);

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471231616



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
##
@@ -301,8 +301,9 @@ public void writeByteArray(Object input, DataOutputStream 
dataOutputStream,
   }
   if (this.carbonDimension.getDataType() == DataTypes.STRING
   && value.length > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException("Dataload failed, String size 
cannot exceed "
-+ CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " 
bytes");
+
logHolder.setReason(String.format(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE,

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471231649



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
##
@@ -330,21 +331,35 @@ public void writeByteArray(Object input, DataOutputStream 
dataOutputStream,
 }
   }
 
+  private byte[] getNullForBytes(byte[] value) {

Review comment:
   removed.





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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471231426



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
##
@@ -194,11 +194,10 @@ class VarcharDataTypesBasicTestCase extends QueryTest 
with BeforeAndAfterEach wi
 // query should pass
 checkAnswer(sql("select * from testlongstring"),
   Seq(Row(1, "ab", "cool"), Row(1, "ab1", longChar), Row(1, "abc", 
longChar)))
-// insert long string should fail as unset is done

Review comment:
   As discussed this change is required because we are setting the bad 
record action as force when we are creating Instance of TestQueryExecutor.





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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471230945



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -154,13 +153,44 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
 sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
 sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
-intercept[Exception] {
-  sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
-}
+checkAnswer(sql("select count(*) from load32000chardata"), Seq(Row(3)))
+// String whilch length greater than 32000 will be considered as bad 
record and will be inserted as null in table
+sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(3)))
+checkAnswer(sql("select * from load32000chardata_dup where mes1=3"), 
Seq(Row("32000", null, 3)))
 sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata_dup 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(6)))
+// Update strings of length greater than 32000 will invalidate the whole 
row.
+sql("update load32000chardata_dup set(load32000chardata_dup.dim2)=(select 
concat(load32000chardata.dim2,'') " +
+  "from load32000chardata where load32000chardata.mes1=3) where 
load32000chardata_dup.mes1=3").show()
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(6)))
+checkAnswer(sql("select * from load32000chardata_dup where mes1=3"), 
Seq(Row("32000", null, 3), Row("32000", null, 3)))
+
+val longChar: String = RandomStringUtils.randomAlphabetic(33000)
+// BAD_RECORD_ACTION = "REDIRECT"
+CarbonProperties.getInstance()
+.addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, 
"REDIRECT");

Review comment:
   Added the separate test case for all the bad record actions and checked 
the redirect value from file.





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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471231039



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -154,13 +153,44 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
 sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
 sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
-intercept[Exception] {
-  sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
-}
+checkAnswer(sql("select count(*) from load32000chardata"), Seq(Row(3)))
+// String whilch length greater than 32000 will be considered as bad 
record and will be inserted as null in table
+sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(3)))
+checkAnswer(sql("select * from load32000chardata_dup where mes1=3"), 
Seq(Row("32000", null, 3)))
 sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata_dup 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(6)))
+// Update strings of length greater than 32000 will invalidate the whole 
row.
+sql("update load32000chardata_dup set(load32000chardata_dup.dim2)=(select 
concat(load32000chardata.dim2,'') " +
+  "from load32000chardata where load32000chardata.mes1=3) where 
load32000chardata_dup.mes1=3").show()
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(6)))
+checkAnswer(sql("select * from load32000chardata_dup where mes1=3"), 
Seq(Row("32000", null, 3), Row("32000", null, 3)))
+
+val longChar: String = RandomStringUtils.randomAlphabetic(33000)
+// BAD_RECORD_ACTION = "REDIRECT"
+CarbonProperties.getInstance()
+.addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, 
"REDIRECT");
+sql(s"insert into load32000chardata_dup values('32000', '$longChar', 3)")
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(6)))
+checkAnswer(sql("select * from load32000chardata_dup where mes1=3"), 
Seq(Row("32000", null, 3), Row("32000", null, 3)))
+
+// BAD_RECORD_ACTION = "IGNORE"
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "IGNORE");
+sql(s"insert into load32000chardata_dup values('32000', '$longChar', 3)")
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(6)))
+checkAnswer(sql("select * from load32000chardata_dup where mes1=3"), 
Seq(Row("32000", null, 3), Row("32000", null, 3)))
+
+// BAD_RECORD_ACTION = "FAIL"
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FAIL");
 intercept[Exception] {
-  sql("update load32000chardata_dup 
set(load32000chardata_dup.dim2)=(select concat(load32000chardata.dim2,'') 
from load32000chardata)").show()
+  sql(s"insert into load32000chardata_dup values('32000', '$longChar', 3)")
 }
+checkAnswer(sql("select count(*) from load32000chardata_dup"), Seq(Row(6)))
+checkAnswer(sql("select * from load32000chardata_dup where mes1=3"), 
Seq(Row("32000", null, 3), Row("32000", null, 3)))
+CarbonProperties.getInstance()

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471230570



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -154,13 +153,44 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
 sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
 sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
-intercept[Exception] {
-  sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()

Review comment:
   As discussed we are setting the bad record action as force when we are 
creating the Instance of TestQueryExecutor. It is not done by other test cases.

##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -154,13 +153,44 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
 sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
 sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
-intercept[Exception] {
-  sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
-}
+checkAnswer(sql("select count(*) from load32000chardata"), Seq(Row(3)))
+// String whilch length greater than 32000 will be considered as bad 
record and will be inserted as null in table

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471229741



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -20,17 +20,16 @@ package 
org.apache.carbondata.integration.spark.testsuite.dataload
 import java.math.BigDecimal
 
 import scala.collection.mutable.ArrayBuffer
-

Review comment:
   done

##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -20,17 +20,16 @@ package 
org.apache.carbondata.integration.spark.testsuite.dataload
 import java.math.BigDecimal
 
 import scala.collection.mutable.ArrayBuffer
-
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterEach
-

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-16 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r471229623



##
File path: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##
@@ -2456,4 +2456,10 @@ private CarbonCommonConstants() {
* property which defines the insert stage flow
*/
   public static final String IS_INSERT_STAGE = "is_insert_stage";
+
+  public static final String STRING_LENGTH_EXCEEDED_MESSAGE =
+  "Record %s of column %s exceeded " + MAX_CHARS_PER_COLUMN_DEFAULT +
+  " bytes. Please consider long string data type.";

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r464911706



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -154,13 +154,18 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
 sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
 sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
-intercept[Exception] {
-  sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
-}
+checkAnswer(sql("select count(*) from load32000chardata"), Seq(Row(3)))

Review comment:
   It was handled for all types of action. In case of "FORCE" action, I am 
just converting long string to null. Now I have added testcases for all those 
types.





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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r464910778



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
##
@@ -82,21 +83,25 @@ public Object convert(Object value, BadRecordLogHolder 
logHolder)
   .getBytesBasedOnDataTypeForNoDictionaryColumn(dimensionValue, 
dataType, dateFormat);
   if (dataType == DataTypes.STRING
   && parsedValue.length > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException(String.format(
-"Dataload failed, String size cannot exceed %d bytes,"
-+ " please consider long string data type",
-CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT));
+
logHolder.setReason(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE);
+String badRecordAction = CarbonProperties.getInstance()
+.getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION);
+if (badRecordAction.equalsIgnoreCase("FORCE")) {

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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r464908207



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
##
@@ -75,11 +75,10 @@ object CarbonScalaUtil {
 carbonLoadModel.getBinaryDecoder)
 } catch {
   case e: Exception =>
-if 
(e.getMessage.startsWith(FieldConverter.stringLengthExceedErrorMsg)) {
-  val msg = s"Column 
${carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-.getCreateOrderColumn.get(idx).getColName} is too long," +
-s" consider to use 'long_string_columns' table property."
-  LOGGER.error(msg, e)
+if 
(e.getMessage.startsWith(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE)) 
{
+  val msg = 
CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE.format(row,
+  
carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getCreateOrderColumn
+.get(idx).getColName)

Review comment:
   done.

##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -170,21 +175,22 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("drop table if exists load32000bytes")
 sql("create table load32000bytes(name string) STORED AS carbondata")
 sql("insert into table load32000bytes select 'aaa'")
+checkAnswer(sql("select count(*) from load32000bytes"), Seq(Row(1)))
 
-assert(intercept[Exception] {
-  sql(s"load data local inpath '$testdata' into table load32000bytes 
OPTIONS ('FILEHEADER'='name')")
-}.getMessage.contains("DataLoad failure: Dataload failed, String size 
cannot exceed 32000 bytes"))
+// Below load will be inserted as null because Strings greater than 32000 
is bad record.
+sql(s"load data local inpath '$testdata' into table load32000bytes OPTIONS 
('FILEHEADER'='name')")
+checkAnswer(sql("select count(*) from load32000bytes"), Seq(Row(2)))
+checkAnswer(sql("select * from load32000bytes"), Seq(Row("aaa"), 
Row(null)))
 
 val source = scala.io.Source.fromFile(testdata, 
CarbonCommonConstants.DEFAULT_CHARSET)
 val data = source.mkString
 
+// Insert will throw exception as it is without converter step.
 intercept[Exception] {
   sql(s"insert into load32000bytes values('$data')")
 }
 
-intercept[Exception] {
-  sql(s"update load32000bytes set(name)= ('$data')").show()
-}
+sql(s"update load32000bytes set(name)= ('$data')").show()

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:
us...@infra.apache.org