[CARBONDATA-1751] Modify sys.err to AnalysisException when uses run related operation except IUD,compaction and alter
carbon printout improper error message, for example, it printout system error when users run create table with the same column name, but it should printout related exception information So we modify sys.error method to AnalysisException when uses run related operation except IUD,compaction and alter This closes #1515 Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/2a12938b Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/2a12938b Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/2a12938b Branch: refs/heads/fgdatamap Commit: 2a12938b545cca5e3c09396dd68393ea615038fa Parents: 1b8d348 Author: xubo245 <601450...@qq.com> Authored: Fri Nov 17 10:48:31 2017 +0800 Committer: Jacky Li <jacky.li...@qq.com> Committed: Sat Nov 18 16:24:23 2017 +0800 ---------------------------------------------------------------------- .../command/CarbonTableSchemaCommonSuite.scala | 72 ++++++++++++++++++++ .../org/apache/carbondata/api/CarbonStore.scala | 4 +- .../carbondata/spark/util/CommonUtil.scala | 17 ++--- .../spark/util/DataTypeConverterUtil.scala | 4 +- .../catalyst/AbstractCarbonSparkSQLParser.scala | 3 +- .../spark/sql/catalyst/CarbonDDLSqlParser.scala | 4 +- .../command/carbonTableSchemaCommon.scala | 10 +-- .../apache/spark/sql/util/CarbonException.scala | 24 +++++++ .../sql/parser/CarbonSpark2SqlParser.scala | 2 +- .../apache/spark/util/CarbonCommandSuite.scala | 2 +- 10 files changed, 124 insertions(+), 18 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark-common-test/src/test/scala/org/apache/spark/sql/execution/command/CarbonTableSchemaCommonSuite.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common-test/src/test/scala/org/apache/spark/sql/execution/command/CarbonTableSchemaCommonSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/spark/sql/execution/command/CarbonTableSchemaCommonSuite.scala new file mode 100644 index 0000000..67dfa8f --- /dev/null +++ b/integration/spark-common-test/src/test/scala/org/apache/spark/sql/execution/command/CarbonTableSchemaCommonSuite.scala @@ -0,0 +1,72 @@ +/* + * 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.spark.sql.execution.command + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.test.util.QueryTest +import org.junit.Assert +import org.scalatest.BeforeAndAfterAll + +class CarbonTableSchemaCommonSuite extends QueryTest with BeforeAndAfterAll { + + test("Creating table: Duplicate dimensions found with name, it should throw AnalysisException") { + sql("DROP TABLE IF EXISTS carbon_table") + try { + sql( + s""" + | CREATE TABLE carbon_table( + | BB INT, bb char(10) + | ) + | STORED BY 'carbondata' + """.stripMargin) + Assert.assertTrue(false) + } catch { + case _: AnalysisException => Assert.assertTrue(true) + case _: Exception => Assert.assertTrue(false) + } finally { + sql("DROP TABLE IF EXISTS carbon_table") + } + } + + test("Altering table: Duplicate column found with name, it should throw RuntimeException") { + sql("DROP TABLE IF EXISTS carbon_table") + sql( + s""" + | CREATE TABLE if not exists carbon_table( + | BB INT, cc char(10) + | ) + | STORED BY 'carbondata' + """.stripMargin) + + try { + sql( + s""" + | alter TABLE carbon_table add columns( + | bb char(10) + ) + """.stripMargin) + Assert.assertTrue(false) + } catch { + case _: RuntimeException => Assert.assertTrue(true) + case _: Exception => Assert.assertTrue(false) + } finally { + sql("DROP TABLE IF EXISTS carbon_table") + } + } + +} http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala index 6c2490e..73325a6 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.types.TimestampType +import org.apache.spark.sql.util.CarbonException import org.apache.carbondata.common.logging.LogServiceFactory import org.apache.carbondata.core.constants.CarbonCommonConstants @@ -57,7 +58,8 @@ object CarbonStore { val lim = Integer.parseInt(limitLoads) loadMetadataDetailsSortedArray = loadMetadataDetailsSortedArray.slice(0, lim) } catch { - case _: NumberFormatException => sys.error(s" Entered limit is not a valid Number") + case _: NumberFormatException => + CarbonException.analysisException("Entered limit is not a valid Number") } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala index a922a07..bed57bd 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.{Row, RowFactory} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.execution.command.{ColumnProperty, Field, PartitionerField} import org.apache.spark.sql.types.{MetadataBuilder, StringType} +import org.apache.spark.sql.util.CarbonException import org.apache.spark.util.FileUtils import org.apache.carbondata.common.logging.LogServiceFactory @@ -662,19 +663,19 @@ object CommonUtil { val maxColumnsInt = getMaxColumnValue(maxColumns) if (maxColumnsInt != null) { if (columnCountInSchema >= maxColumnsInt) { - sys.error(s"csv headers should be less than the max columns: $maxColumnsInt") + CarbonException.analysisException( + s"csv headers should be less than the max columns: $maxColumnsInt") } else if (maxColumnsInt > CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) { - sys.error(s"max columns cannot be greater than the threshold value: ${ - CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING - }") + CarbonException.analysisException( + s"max columns cannot be greater than the threshold value: " + + s"${CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING}") } else { maxNumberOfColumnsForParsing = maxColumnsInt } } else if (columnCountInSchema >= CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) { - sys.error(s"csv header columns should be less than max threashold: ${ - CSVInputFormat - .THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING - }") + CarbonException.analysisException( + s"csv header columns should be less than max threashold: " + + s"${CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING}") } else if (columnCountInSchema >= CSVInputFormat.DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) { maxNumberOfColumnsForParsing = columnCountInSchema + 1 } else { http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala index 0969fdf..6673e18 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala @@ -17,6 +17,8 @@ package org.apache.carbondata.spark.util +import org.apache.spark.sql.util.CarbonException + import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes} import org.apache.carbondata.format.{DataType => ThriftDataType} @@ -75,7 +77,7 @@ object DataTypeConverterUtil { } else if (others != null && others.startsWith("varchar")) { DataTypes.STRING } else { - sys.error(s"Unsupported data type: $dataType") + CarbonException.analysisException(s"Unsupported data type: $dataType") } } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/AbstractCarbonSparkSQLParser.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/AbstractCarbonSparkSQLParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/AbstractCarbonSparkSQLParser.scala index fba7976..2dc2d4e 100644 --- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/AbstractCarbonSparkSQLParser.scala +++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/AbstractCarbonSparkSQLParser.scala @@ -24,6 +24,7 @@ import scala.util.parsing.combinator.PackratParsers import scala.util.parsing.input.CharArrayReader.EofCh import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.util.CarbonException private[sql] abstract class AbstractCarbonSparkSQLParser extends StandardTokenParsers with PackratParsers { @@ -33,7 +34,7 @@ private[sql] abstract class AbstractCarbonSparkSQLParser initLexical phrase(start)(new lexical.Scanner(input)) match { case Success(plan, _) => plan - case failureOrError => sys.error(failureOrError.toString) + case failureOrError => CarbonException.analysisException(failureOrError.toString) } } /* One time initialization of lexical.This avoid reinitialization of lexical in parse method */ http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/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 0e77e8c..b4cde9b 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 @@ -28,8 +28,10 @@ import scala.util.matching.Regex import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.util.CarbonException import org.apache.spark.util.PartitionUtils import org.apache.carbondata.common.constants.LoggerAction @@ -781,7 +783,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser { } if (remainingNodes.nonEmpty) { - sys.error( + CarbonException.analysisException( s"""Unhandled clauses: |You are likely trying to use an unsupported carbon feature."""".stripMargin) } http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala index 9c927aa..d64544d 100644 --- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala +++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala @@ -26,6 +26,7 @@ import scala.collection.mutable.Map import org.apache.spark.SparkContext import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.util.CarbonException import org.apache.carbondata.common.logging.LogServiceFactory import org.apache.carbondata.core.constants.CarbonCommonConstants @@ -530,7 +531,7 @@ class TableNewProcessor(cm: TableModel) { s"Validation failed for Create/Alter Table Operation " + s"for ${ cm.databaseName }.${ cm.tableName }" + s"Duplicate column found with name: $name") - sys.error(s"Duplicate dimensions found with name: $name") + CarbonException.analysisException(s"Duplicate dimensions found with name: $name") } } @@ -600,12 +601,12 @@ class TableNewProcessor(cm: TableModel) { } else { LOGGER.error(s"Bucket field must be dimension column and " + s"should not be measure or complex column: ${colSchema.getColumnName}") - sys.error(s"Bucket field must be dimension column and " + + CarbonException.analysisException(s"Bucket field must be dimension column and " + s"should not be measure or complex column: ${colSchema.getColumnName}") } case _ => LOGGER.error(s"Bucket field is not present in table columns") - sys.error(s"Bucket field is not present in table columns") + CarbonException.analysisException(s"Bucket field is not present in table columns") } } tableSchema.setBucketingInfo( @@ -653,7 +654,8 @@ class TableNewProcessor(cm: TableModel) { }) if (!found) { - sys.error(s"column $colForGrouping is not present in Field list") + CarbonException.analysisException( + s"column $colForGrouping is not present in Field list") } }) }) http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark-common/src/main/scala/org/apache/spark/sql/util/CarbonException.scala ---------------------------------------------------------------------- diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/util/CarbonException.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/util/CarbonException.scala new file mode 100644 index 0000000..9fd7099 --- /dev/null +++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/util/CarbonException.scala @@ -0,0 +1,24 @@ +/* + * 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.spark.sql.util + + import org.apache.spark.sql.AnalysisException + + object CarbonException { + def analysisException(message: String): Nothing = throw new AnalysisException(message) + } http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala ---------------------------------------------------------------------- diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala index 7a637f7..c8ea4ac 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala @@ -479,7 +479,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser { match { case Success(field, _) => field.asInstanceOf[Field] case failureOrError => throw new MalformedCarbonCommandException( - s"Unsupported data type: $col.getType") + s"Unsupported data type: ${ col.dataType }") } // the data type of the decimal type will be like decimal(10,0) // so checking the start of the string and taking the precision and scale. http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a12938b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala ---------------------------------------------------------------------- diff --git a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala index 9f89226..9e20cdd 100644 --- a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala +++ b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala @@ -100,7 +100,7 @@ class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll { test("delete segment by id") { DeleteSegmentById.main(Array(s"${location}", "carbon_table", "0")) - assert(!CarbonStore.isSegmentValid("default", "carbon_table",location, "0")) + assert(!CarbonStore.isSegmentValid("default", "carbon_table", location, "0")) } test("delete segment by date") {