Repository: incubator-carbondata
Updated Branches:
  refs/heads/master fa7421a18 -> 58cd9339e


[CARBONDATA-400]
* Problem: When the number of characters in a column exceeds 100000 characters 
whole string appears in beeline with exception.

Analysis: In univocity csv parser settings , the maximum number of characters 
per column is 100000 and when it exceeds that limit, TextparsingException is 
thrown with
the complete string as error in beeline during data load.

Fix: Now a proper error message is displayed in beeline and complete error 
messages and parser settings details will be present in logs.
Impact area: Data loading with more than 100000 characters in a single column.


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: 
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/c99cf06d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/c99cf06d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/c99cf06d

Branch: refs/heads/master
Commit: c99cf06de469ac97c3d7269377394d76190554e7
Parents: fa7421a
Author: Akash R Nilugal <akashnilu...@gmail.com>
Authored: Mon Dec 5 15:16:10 2016 +0530
Committer: ravipesala <ravi.pes...@gmail.com>
Committed: Thu Apr 6 15:33:53 2017 +0530

----------------------------------------------------------------------
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |  3 +++
 .../spark/util/GlobalDictionaryUtil.scala       | 25 +++++++++++++++++---
 .../processing/csvload/CSVInputFormat.java      |  1 +
 3 files changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c99cf06d/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
index 74531d3..ea71ea1 100644
--- 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
+++ 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
@@ -27,6 +27,7 @@ import scala.collection.mutable.ArrayBuffer
 import scala.util.control.Breaks.{break, breakable}
 
 import au.com.bytecode.opencsv.CSVReader
+import com.univocity.parsers.common.TextParsingException
 import org.apache.commons.lang3.{ArrayUtils, StringUtils}
 import org.apache.spark._
 import org.apache.spark.rdd.RDD
@@ -307,6 +308,8 @@ class CarbonBlockDistinctValuesCombineRDD(
       }
       
CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordLoadCsvfilesToDfTime()
     } catch {
+      case txe: TextParsingException =>
+        throw txe
       case ex: Exception =>
         LOGGER.error(ex)
         throw ex

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c99cf06d/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index 5cb493c..aeb387a 100644
--- 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -32,7 +32,7 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
-import org.apache.spark.Accumulator
+import org.apache.spark.{Accumulator, SparkException}
 import org.apache.spark.rdd.{NewHadoopRDD, RDD}
 import org.apache.spark.sql._
 import org.apache.spark.sql.types.{StringType, StructField, StructType}
@@ -784,9 +784,28 @@ object GlobalDictionaryUtil {
       }
     } catch {
       case ex: Exception =>
-        LOGGER.error(ex, "generate global dictionary failed")
-        throw ex
+        ex match {
+          case spx: SparkException =>
+            LOGGER.error(spx, "generate global dictionary failed")
+            throw new Exception("generate global dictionary failed, " +
+                                trimErrorMessage(spx.getMessage))
+          case _ =>
+            LOGGER.error(ex, "generate global dictionary failed")
+            throw ex
+        }
+    }
+  }
+
+  // Get proper error message of TextParsingException
+  def trimErrorMessage(input: String): String = {
+    var errorMessage: String = null
+    if (input != null) {
+      if (input.split("Hint").length > 0 &&
+          input.split("Hint")(0).split("TextParsingException: ").length > 1) {
+        errorMessage = input.split("Hint")(0).split("TextParsingException: 
")(1)
+      }
     }
+    errorMessage
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c99cf06d/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
----------------------------------------------------------------------
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
 
b/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
index 7545fe5..1f7d403 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
@@ -219,6 +219,7 @@ public class CSVInputFormat extends 
FileInputFormat<NullWritable, StringArrayWri
       parserSettings.setIgnoreLeadingWhitespaces(false);
       parserSettings.setIgnoreTrailingWhitespaces(false);
       parserSettings.setSkipEmptyLines(false);
+      parserSettings.setMaxCharsPerColumn(100000);
       // TODO get from csv file.
       parserSettings.setMaxColumns(1000);
       parserSettings.getFormat().setQuote(job.get(QUOTE, 
QUOTE_DEFAULT).charAt(0));

Reply via email to