carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gvram...@apache.org
Subject [03/22] incubator-carbondata git commit: IUD checkstyle and scalasytle fixes
Date Fri, 06 Jan 2017 13:57:03 GMT
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/002279ec/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index d33d3df..518ab05 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -65,6 +65,16 @@ class CarbonSqlParser() extends CarbonDDLSqlParser {
     }
   }
 
+  /**
+   * This will convert key word to regular expression.
+   *
+   * @param keys
+   * @return
+   */
+  private def carbonKeyWord(keys: String) = {
+    ("(?i)" + keys).r
+  }
+
   override protected lazy val start: Parser[LogicalPlan] = explainPlan | startCommand
 
   protected lazy val startCommand: Parser[LogicalPlan] =
@@ -110,6 +120,19 @@ class CarbonSqlParser() extends CarbonDDLSqlParser {
         DropDatabase(dbName, isCascade, dropDbSql)
     }
 
+  private def reorderDimensions(dims: Seq[Field]): Seq[Field] = {
+    var complexDimensions: Seq[Field] = Seq()
+    var dimensions: Seq[Field] = Seq()
+    dims.foreach { dimension =>
+      dimension.dataType.getOrElse("NIL") match {
+        case "Array" => complexDimensions = complexDimensions :+ dimension
+        case "Struct" => complexDimensions = complexDimensions :+ dimension
+        case _ => dimensions = dimensions :+ dimension
+      }
+    }
+    dimensions ++ complexDimensions
+  }
+
   protected lazy val alterTable: Parser[LogicalPlan] =
     ALTER ~> TABLE ~> restInput ^^ {
       case statement =>
@@ -147,6 +170,14 @@ class CarbonSqlParser() extends CarbonDDLSqlParser {
       }
   }
 
+  private def getScaleAndPrecision(dataType: String): (Int, Int) = {
+    val m: Matcher = Pattern.compile("^decimal\\(([^)]+)\\)").matcher(dataType)
+    m.find()
+    val matchedString: String = m.group(1)
+    val scaleAndPrecision = matchedString.split(",")
+    (Integer.parseInt(scaleAndPrecision(0).trim), Integer.parseInt(scaleAndPrecision(1).trim))
+  }
+
   /**
    * This function will traverse the tree and logical plan will be formed using that.
    *
@@ -345,6 +376,510 @@ class CarbonSqlParser() extends CarbonDDLSqlParser {
     }
   }
 
+  /**
+   * This will prepate the Model from the Tree details.
+   *
+   * @param ifNotExistPresent
+   * @param dbName
+   * @param tableName
+   * @param fields
+   * @param partitionCols
+   * @param tableProperties
+   * @return
+   */
+  protected def prepareTableModel(ifNotExistPresent: Boolean, dbName: Option[String]
+      , tableName: String, fields: Seq[Field],
+      partitionCols: Seq[PartitionerField],
+      tableProperties: Map[String, String]): TableModel
+  = {
+
+    fields.zipWithIndex.foreach { x =>
+      x._1.schemaOrdinal = x._2
+    }
+    val (dims: Seq[Field], noDictionaryDims: Seq[String]) = extractDimColsAndNoDictionaryFields(
+      fields, tableProperties)
+    if (dims.isEmpty) {
+      throw new MalformedCarbonCommandException(s"Table ${
+        dbName.getOrElse(
+          CarbonCommonConstants.DATABASE_DEFAULT_NAME)
+      }.$tableName"
+                                                +
+                                                " can not be created without key columns. Please " +
+                                                "use DICTIONARY_INCLUDE or " +
+                                                "DICTIONARY_EXCLUDE to set at least one key " +
+                                                "column " +
+                                                "if all specified columns are numeric types")
+    }
+    val msrs: Seq[Field] = extractMsrColsFromFields(fields, tableProperties)
+
+    // column properties
+    val colProps = extractColumnProperties(fields, tableProperties)
+    // get column groups configuration from table properties.
+    val groupCols: Seq[String] = updateColumnGroupsInField(tableProperties,
+      noDictionaryDims, msrs, dims)
+
+    // get no inverted index columns from table properties.
+    val noInvertedIdxCols = extractNoInvertedIndexColumns(fields, tableProperties)
+
+    // validate the tableBlockSize from table properties
+    CommonUtil.validateTableBlockSize(tableProperties)
+
+    TableModel(
+      ifNotExistPresent,
+      dbName.getOrElse(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+      dbName,
+      tableName,
+      tableProperties,
+      reorderDimensions(dims.map(f => normalizeType(f)).map(f => addParent(f))),
+      msrs.map(f => normalizeType(f)),
+      Option(noDictionaryDims),
+      Option(noInvertedIdxCols),
+      groupCols,
+      Some(colProps))
+  }
+
+  /**
+   * Extract the column groups configuration from table properties.
+   * Based on this Row groups of fields will be determined.
+   *
+   * @param tableProperties
+   * @return
+   */
+  protected def updateColumnGroupsInField(tableProperties: Map[String, String],
+      noDictionaryDims: Seq[String],
+      msrs: Seq[Field],
+      dims: Seq[Field]): Seq[String] = {
+    if (tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS).isDefined) {
+
+      var splittedColGrps: Seq[String] = Seq[String]()
+      val nonSplitCols: String = tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS).get
+
+      // row groups will be specified in table properties like -> "(col1,col2),(col3,col4)"
+      // here first splitting the value by () . so that the above will be splitted into 2 strings.
+      // [col1,col2] [col3,col4]
+      val m: Matcher = Pattern.compile("\\(([^)]+)\\)").matcher(nonSplitCols)
+      while (m.find()) {
+        val oneGroup: String = m.group(1)
+        CommonUtil.validateColumnGroup(oneGroup, noDictionaryDims, msrs, splittedColGrps, dims)
+        val arrangedColGrp = rearrangedColumnGroup(oneGroup, dims)
+        splittedColGrps :+= arrangedColGrp
+      }
+      // This will  be furthur handled.
+      CommonUtil.arrangeColGrpsInSchemaOrder(splittedColGrps, dims)
+    } else {
+      null
+    }
+  }
+
+  def rearrangedColumnGroup(colGroup: String, dims: Seq[Field]): String = {
+    // if columns in column group is not in schema order than arrange it in schema order
+    var colGrpFieldIndx: Seq[Int] = Seq[Int]()
+    colGroup.split(',').map(_.trim).foreach { x =>
+      dims.zipWithIndex.foreach { dim =>
+        if (dim._1.column.equalsIgnoreCase(x)) {
+          colGrpFieldIndx :+= dim._2
+        }
+      }
+    }
+    // sort it
+    colGrpFieldIndx = colGrpFieldIndx.sorted
+    // check if columns in column group is in schema order
+    if (!checkIfInSequence(colGrpFieldIndx)) {
+      throw new MalformedCarbonCommandException("Invalid column group:" + colGroup)
+    }
+    def checkIfInSequence(colGrpFieldIndx: Seq[Int]): Boolean = {
+      for (i <- 0 until (colGrpFieldIndx.length - 1)) {
+        if ((colGrpFieldIndx(i + 1) - colGrpFieldIndx(i)) != 1) {
+          throw new MalformedCarbonCommandException(
+            "Invalid column group,column in group should be contiguous as per schema.")
+        }
+      }
+      true
+    }
+    val colGrpNames: StringBuilder = StringBuilder.newBuilder
+    for (i <- colGrpFieldIndx.indices) {
+      colGrpNames.append(dims(colGrpFieldIndx(i)).column)
+      if (i < (colGrpFieldIndx.length - 1)) {
+        colGrpNames.append(",")
+      }
+    }
+    colGrpNames.toString()
+  }
+
+  /**
+   * For getting the partitioner Object
+   *
+   * @param partitionCols
+   * @param tableProperties
+   * @return
+   */
+  protected def getPartitionerObject(partitionCols: Seq[PartitionerField],
+      tableProperties: Map[String, String]):
+  Option[Partitioner] = {
+
+    // by default setting partition class empty.
+    // later in table schema it is setting to default value.
+    var partitionClass: String = ""
+    var partitionCount: Int = 1
+    var partitionColNames: Array[String] = Array[String]()
+    if (tableProperties.get(CarbonCommonConstants.PARTITIONCLASS).isDefined) {
+      partitionClass = tableProperties.get(CarbonCommonConstants.PARTITIONCLASS).get
+    }
+
+    if (tableProperties.get(CarbonCommonConstants.PARTITIONCOUNT).isDefined) {
+      try {
+        partitionCount = tableProperties.get(CarbonCommonConstants.PARTITIONCOUNT).get.toInt
+      } catch {
+        case e: Exception => // no need to do anything.
+      }
+    }
+
+    partitionCols.foreach(col =>
+      partitionColNames :+= col.partitionColumn
+    )
+
+    // this means user has given partition cols list
+    if (!partitionColNames.isEmpty) {
+      return Option(Partitioner(partitionClass, partitionColNames, partitionCount, null))
+    }
+    // if partition cols are not given then no need to do partition.
+    None
+  }
+
+  protected def extractColumnProperties(fields: Seq[Field], tableProperties: Map[String, String]):
+  java.util.Map[String, java.util.List[ColumnProperty]] = {
+    val colPropMap = new java.util.HashMap[String, java.util.List[ColumnProperty]]()
+    fields.foreach { field =>
+      if (field.children.isDefined && field.children.get != null) {
+        fillAllChildrenColumnProperty(field.column, field.children, tableProperties, colPropMap)
+      } else {
+        fillColumnProperty(None, field.column, tableProperties, colPropMap)
+      }
+    }
+    colPropMap
+  }
+
+  protected def fillAllChildrenColumnProperty(parent: String, fieldChildren: Option[List[Field]],
+      tableProperties: Map[String, String],
+      colPropMap: java.util.HashMap[String, java.util.List[ColumnProperty]]) {
+    fieldChildren.foreach(fields => {
+      fields.foreach(field => {
+        fillColumnProperty(Some(parent), field.column, tableProperties, colPropMap)
+      }
+      )
+    }
+    )
+  }
+
+  protected def fillColumnProperty(parentColumnName: Option[String],
+      columnName: String,
+      tableProperties: Map[String, String],
+      colPropMap: java.util.HashMap[String, java.util.List[ColumnProperty]]) {
+    val (tblPropKey, colProKey) = getKey(parentColumnName, columnName)
+    val colProps = CommonUtil.getColumnProperties(tblPropKey, tableProperties)
+    if (colProps.isDefined) {
+      colPropMap.put(colProKey, colProps.get)
+    }
+  }
+
+  def getKey(parentColumnName: Option[String],
+      columnName: String): (String, String) = {
+    if (parentColumnName.isDefined) {
+      if (columnName == "val") {
+        (parentColumnName.get, parentColumnName.get + "." + columnName)
+      } else {
+        (parentColumnName.get + "." + columnName, parentColumnName.get + "." + columnName)
+      }
+    } else {
+      (columnName, columnName)
+    }
+  }
+
+  /**
+   * This will extract the no inverted columns fields.
+   * By default all dimensions use inverted index.
+   *
+   * @param fields
+   * @param tableProperties
+   * @return
+   */
+  protected def extractNoInvertedIndexColumns(fields: Seq[Field],
+      tableProperties: Map[String, String]):
+  Seq[String] = {
+    // check whether the column name is in fields
+    var noInvertedIdxColsProps: Array[String] = Array[String]()
+    var noInvertedIdxCols: Seq[String] = Seq[String]()
+
+    if (tableProperties.get("NO_INVERTED_INDEX").isDefined) {
+      noInvertedIdxColsProps =
+        tableProperties.get("NO_INVERTED_INDEX").get.split(',').map(_.trim)
+      noInvertedIdxColsProps
+        .map { noInvertedIdxColProp =>
+          if (!fields.exists(x => x.column.equalsIgnoreCase(noInvertedIdxColProp))) {
+            val errormsg = "NO_INVERTED_INDEX column: " + noInvertedIdxColProp +
+                           " does not exist in table. Please check create table statement."
+            throw new MalformedCarbonCommandException(errormsg)
+          }
+        }
+    }
+    // check duplicate columns and only 1 col left
+    val distinctCols = noInvertedIdxColsProps.toSet
+    // extract the no inverted index columns
+    fields.foreach(field => {
+      if (distinctCols.exists(x => x.equalsIgnoreCase(field.column))) {
+        noInvertedIdxCols :+= field.column
+      }
+    }
+    )
+    noInvertedIdxCols
+  }
+
+  /**
+   * This will extract the Dimensions and NoDictionary Dimensions fields.
+   * By default all string cols are dimensions.
+   *
+   * @param fields
+   * @param tableProperties
+   * @return
+   */
+  protected def extractDimColsAndNoDictionaryFields(fields: Seq[Field],
+      tableProperties: Map[String, String]):
+  (Seq[Field], Seq[String]) = {
+    var dimFields: LinkedHashSet[Field] = LinkedHashSet[Field]()
+    var dictExcludeCols: Array[String] = Array[String]()
+    var noDictionaryDims: Seq[String] = Seq[String]()
+    var dictIncludeCols: Seq[String] = Seq[String]()
+
+    // All excluded cols should be there in create table cols
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).isDefined) {
+      dictExcludeCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
+      dictExcludeCols
+        .map { dictExcludeCol =>
+          if (!fields.exists(x => x.column.equalsIgnoreCase(dictExcludeCol))) {
+            val errormsg = "DICTIONARY_EXCLUDE column: " + dictExcludeCol +
+                           " does not exist in table. Please check create table statement."
+            throw new MalformedCarbonCommandException(errormsg)
+          } else {
+            val dataType = fields.find(x =>
+              x.column.equalsIgnoreCase(dictExcludeCol)).get.dataType.get
+            if (isComplexDimDictionaryExclude(dataType)) {
+              val errormsg = "DICTIONARY_EXCLUDE is unsupported for complex datatype column: " +
+                             dictExcludeCol
+              throw new MalformedCarbonCommandException(errormsg)
+            } else if (!isStringAndTimestampColDictionaryExclude(dataType)) {
+              val errorMsg = "DICTIONARY_EXCLUDE is unsupported for " + dataType.toLowerCase() +
+                             " data type column: " + dictExcludeCol
+              throw new MalformedCarbonCommandException(errorMsg)
+            }
+          }
+        }
+    }
+    // All included cols should be there in create table cols
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).isDefined) {
+      dictIncludeCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).get.split(",").map(_.trim)
+      dictIncludeCols.map { distIncludeCol =>
+        if (!fields.exists(x => x.column.equalsIgnoreCase(distIncludeCol.trim))) {
+          val errormsg = "DICTIONARY_INCLUDE column: " + distIncludeCol.trim +
+                         " does not exist in table. Please check create table statement."
+          throw new MalformedCarbonCommandException(errormsg)
+        }
+      }
+    }
+
+    // include cols should contain exclude cols
+    dictExcludeCols.foreach { dicExcludeCol =>
+      if (dictIncludeCols.exists(x => x.equalsIgnoreCase(dicExcludeCol))) {
+        val errormsg = "DICTIONARY_EXCLUDE can not contain the same column: " + dicExcludeCol +
+                       " with DICTIONARY_INCLUDE. Please check create table statement."
+        throw new MalformedCarbonCommandException(errormsg)
+      }
+    }
+
+    // by default consider all String cols as dims and if any dictionary exclude is present then
+    // add it to noDictionaryDims list. consider all dictionary excludes/include cols as dims
+    fields.foreach(field => {
+
+      if (dictExcludeCols.toSeq.exists(x => x.equalsIgnoreCase(field.column))) {
+        if (DataTypeUtil.getDataType(field.dataType.get.toUpperCase()) != DataType.TIMESTAMP) {
+          noDictionaryDims :+= field.column
+        }
+        dimFields += field
+      } else if (dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
+        dimFields += (field)
+      } else if (isDetectAsDimentionDatatype(field.dataType.get)) {
+        dimFields += (field)
+      }
+    }
+    )
+
+    (dimFields.toSeq, noDictionaryDims)
+  }
+
+  /**
+   * It fills non string dimensions in dimFields
+   */
+  def fillNonStringDimension(dictIncludeCols: Seq[String],
+      field: Field, dimFields: LinkedHashSet[Field]) {
+    var dictInclude = false
+    if (dictIncludeCols.nonEmpty) {
+      dictIncludeCols.foreach(dictIncludeCol =>
+        if (field.column.equalsIgnoreCase(dictIncludeCol)) {
+          dictInclude = true
+        })
+    }
+    if (dictInclude) {
+      dimFields += field
+    }
+  }
+
+  /**
+   * detect dimention data type
+   *
+   * @param dimensionDatatype
+   */
+  def isDetectAsDimentionDatatype(dimensionDatatype: String): Boolean = {
+    val dimensionType = Array("string", "array", "struct", "timestamp")
+    dimensionType.exists(x => x.equalsIgnoreCase(dimensionDatatype))
+  }
+
+  /**
+   * detects whether complex dimension is part of dictionary_exclude
+   */
+  def isComplexDimDictionaryExclude(dimensionDataType: String): Boolean = {
+    val dimensionType = Array("array", "struct")
+    dimensionType.exists(x => x.equalsIgnoreCase(dimensionDataType))
+  }
+
+  /**
+   * detects whether double or decimal column is part of dictionary_exclude
+   */
+  def isStringAndTimestampColDictionaryExclude(columnDataType: String): Boolean = {
+    val dataTypes = Array("string", "timestamp")
+    dataTypes.exists(x => x.equalsIgnoreCase(columnDataType))
+  }
+
+  /**
+   * Extract the Measure Cols fields. By default all non string cols will be measures.
+   *
+   * @param fields
+   * @param tableProperties
+   * @return
+   */
+  protected def extractMsrColsFromFields(fields: Seq[Field],
+      tableProperties: Map[String, String]): Seq[Field] = {
+    var msrFields: Seq[Field] = Seq[Field]()
+    var dictIncludedCols: Array[String] = Array[String]()
+    var dictExcludedCols: Array[String] = Array[String]()
+
+    // get all included cols
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).isDefined) {
+      dictIncludedCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).get.split(',').map(_.trim)
+    }
+
+    // get all excluded cols
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).isDefined) {
+      dictExcludedCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
+    }
+
+    // by default consider all non string cols as msrs. consider all include/ exclude cols as dims
+    fields.foreach(field => {
+      if (!isDetectAsDimentionDatatype(field.dataType.get)) {
+        if (!dictIncludedCols.exists(x => x.equalsIgnoreCase(field.column)) &&
+            !dictExcludedCols.exists(x => x.equalsIgnoreCase(field.column))) {
+          msrFields :+= field
+        }
+      }
+    })
+
+    msrFields
+  }
+
+  /**
+   * Extract the DbName and table name.
+   *
+   * @param tableNameParts
+   * @return
+   */
+  protected def extractDbNameTableName(tableNameParts: Node): (Option[String], String) = {
+    val (db, tableName) =
+      tableNameParts.getChildren.asScala.map {
+        case Token(part, Nil) => cleanIdentifier(part)
+      } match {
+        case Seq(tableOnly) => (None, tableOnly)
+        case Seq(databaseName, table) => (Some(databaseName), table)
+      }
+
+    (db, tableName)
+  }
+
+  protected def cleanIdentifier(ident: String): String = {
+    ident match {
+      case escapedIdentifier(i) => i
+      case plainIdent => plainIdent
+    }
+  }
+
+  protected def getClauses(clauseNames: Seq[String], nodeList: Seq[ASTNode]): Seq[Option[Node]] = {
+    var remainingNodes = nodeList
+    val clauses = clauseNames.map { clauseName =>
+      val (matches, nonMatches) = remainingNodes.partition(_.getText.toUpperCase == clauseName)
+      remainingNodes = nonMatches ++ (if (matches.nonEmpty) {
+        matches.tail
+      } else {
+        Nil
+      })
+      matches.headOption
+    }
+
+    if (remainingNodes.nonEmpty) {
+      sys.error(
+        s"""Unhandled clauses:
+            |You are likely trying to use an unsupported carbon feature."""".stripMargin)
+    }
+    clauses
+  }
+
+  object Token {
+    /** @return matches of the form (tokenName, children). */
+    def unapply(t: Any): Option[(String, Seq[ASTNode])] = {
+      t match {
+        case t: ASTNode =>
+          CurrentOrigin.setPosition(t.getLine, t.getCharPositionInLine)
+          Some((t.getText,
+            Option(t.getChildren).map(_.asScala.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]]))
+        case _ => None
+      }
+    }
+  }
+
+  /**
+   * Extract the table properties token
+   *
+   * @param node
+   * @return
+   */
+  protected def getProperties(node: Node): Seq[(String, String)] = {
+    node match {
+      case Token("TOK_TABLEPROPLIST", list) =>
+        list.map {
+          case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) =>
+            (unquoteString(key) -> unquoteString(value))
+        }
+    }
+  }
+
+  protected def unquoteString(str: String) = {
+    str match {
+      case singleQuotedString(s) => s.toLowerCase()
+      case doubleQuotedString(s) => s.toLowerCase()
+      case other => other
+    }
+  }
+
   protected lazy val loadDataNew: Parser[LogicalPlan] =
     LOAD ~> DATA ~> opt(LOCAL) ~> INPATH ~> stringLit ~ opt(OVERWRITE) ~
     (INTO ~> TABLE ~> (ident <~ ".").? ~ ident) ~
@@ -361,6 +896,134 @@ class CarbonSqlParser() extends CarbonDDLSqlParser {
           isOverwrite.isDefined)
     }
 
+  private def validateOptions(optionList: Option[List[(String, String)]]): Unit = {
+
+    // validate with all supported options
+    val options = optionList.get.groupBy(x => x._1)
+    val supportedOptions = Seq("DELIMITER", "QUOTECHAR", "FILEHEADER", "ESCAPECHAR", "MULTILINE",
+      "COMPLEX_DELIMITER_LEVEL_1", "COMPLEX_DELIMITER_LEVEL_2", "COLUMNDICT",
+      "SERIALIZATION_NULL_FORMAT", "BAD_RECORDS_LOGGER_ENABLE", "BAD_RECORDS_ACTION",
+      "ALL_DICTIONARY_PATH", "MAXCOLUMNS", "COMMENTCHAR", "USE_KETTLE", "DATEFORMAT"
+    )
+    var isSupported = true
+    val invalidOptions = StringBuilder.newBuilder
+    options.foreach(value => {
+      if (!supportedOptions.exists(x => x.equalsIgnoreCase(value._1))) {
+        isSupported = false
+        invalidOptions.append(value._1)
+      }
+
+    }
+    )
+    if (!isSupported) {
+      val errorMessage = "Error: Invalid option(s): " + invalidOptions.toString()
+      throw new MalformedCarbonCommandException(errorMessage)
+    }
+
+    //  COLUMNDICT and ALL_DICTIONARY_PATH can not be used together.
+    if (options.exists(_._1.equalsIgnoreCase("COLUMNDICT")) &&
+        options.exists(_._1.equalsIgnoreCase("ALL_DICTIONARY_PATH"))) {
+      val errorMessage = "Error: COLUMNDICT and ALL_DICTIONARY_PATH can not be used together" +
+                         " in options"
+      throw new MalformedCarbonCommandException(errorMessage)
+    }
+
+    if (options.exists(_._1.equalsIgnoreCase("MAXCOLUMNS"))) {
+      val maxColumns: String = options.get("maxcolumns").get(0)._2
+      try {
+        maxColumns.toInt
+      } catch {
+        case ex: NumberFormatException =>
+          throw new MalformedCarbonCommandException(
+            "option MAXCOLUMNS can only contain integer values")
+      }
+    }
+
+    // check for duplicate options
+    val duplicateOptions = options filter {
+      case (_, optionlist) => optionlist.size > 1
+    }
+    val duplicates = StringBuilder.newBuilder
+    if (duplicateOptions.nonEmpty) {
+      duplicateOptions.foreach(x => {
+        duplicates.append(x._1)
+      }
+      )
+      val errorMessage = "Error: Duplicate option(s): " + duplicates.toString()
+      throw new MalformedCarbonCommandException(errorMessage)
+    }
+  }
+
+  protected lazy val dbTableIdentifier: Parser[Seq[String]] =
+    (ident <~ ".").? ~ (ident) ^^ {
+      case databaseName ~ tableName =>
+        if (databaseName.isDefined) {
+          Seq(databaseName.get, tableName)
+        } else {
+          Seq(tableName)
+        }
+    }
+
+  protected lazy val loadOptions: Parser[(String, String)] =
+    (stringLit <~ "=") ~ stringLit ^^ {
+      case opt ~ optvalue => (opt.trim.toLowerCase(), optvalue)
+      case _ => ("", "")
+    }
+
+
+  protected lazy val dimCol: Parser[Field] = anyFieldDef
+
+  protected lazy val primitiveTypes =
+    STRING ^^^ "string" | INTEGER ^^^ "integer" |
+    TIMESTAMP ^^^ "timestamp" | NUMERIC ^^^ "numeric" |
+    BIGINT ^^^ "bigint" | SHORT ^^^ "smallint" |
+    INT ^^^ "int" | DOUBLE ^^^ "double" | decimalType
+
+  /**
+   * Matching the decimal(10,0) data type and returning the same.
+   */
+  private lazy val decimalType =
+  DECIMAL ~ ("(" ~> numericLit <~ ",") ~ (numericLit <~ ")") ^^ {
+    case decimal ~ precision ~ scale =>
+      s"$decimal($precision, $scale)"
+  }
+
+  protected lazy val nestedType: Parser[Field] = structFieldType | arrayFieldType |
+                                                 primitiveFieldType
+
+  protected lazy val anyFieldDef: Parser[Field] =
+    (ident | stringLit) ~ ((":").? ~> nestedType) ~ (IN ~> (ident | stringLit)).? ^^ {
+      case e1 ~ e2 ~ e3 =>
+        Field(e1, e2.dataType, Some(e1), e2.children, null, e3)
+    }
+
+  protected lazy val primitiveFieldType: Parser[Field] =
+    (primitiveTypes) ^^ {
+      case e1 =>
+        Field("unknown", Some(e1), Some("unknown"), Some(null))
+    }
+
+  protected lazy val arrayFieldType: Parser[Field] =
+    ((ARRAY ^^^ "array") ~> "<" ~> nestedType <~ ">") ^^ {
+      case e1 =>
+        Field("unknown", Some("array"), Some("unknown"),
+          Some(List(Field("val", e1.dataType, Some("val"),
+            e1.children))))
+    }
+
+  protected lazy val structFieldType: Parser[Field] =
+    ((STRUCT ^^^ "struct") ~> "<" ~> repsep(anyFieldDef, ",") <~ ">") ^^ {
+      case e1 =>
+        Field("unknown", Some("struct"), Some("unknown"), Some(e1))
+    }
+
+  protected lazy val measureCol: Parser[Field] =
+    (ident | stringLit) ~ (INTEGER ^^^ "integer" | NUMERIC ^^^ "numeric" | SHORT ^^^ "smallint" |
+                           BIGINT ^^^ "bigint" | DECIMAL ^^^ "decimal").? ~
+    (AS ~> (ident | stringLit)).? ~ (IN ~> (ident | stringLit)).? ^^ {
+      case e1 ~ e2 ~ e3 ~ e4 => Field(e1, e2, e3, Some(null))
+    }
+
   protected lazy val describeTable: Parser[LogicalPlan] =
     ((DESCRIBE | DESC) ~> opt(EXTENDED | FORMATTED)) ~ (ident <~ ".").? ~ ident ^^ {
       case ef ~ db ~ tbl =>
@@ -378,6 +1041,109 @@ class CarbonSqlParser() extends CarbonDDLSqlParser {
         }
     }
 
+  private def normalizeType(field: Field): Field = {
+    val dataType = field.dataType.getOrElse("NIL")
+    dataType match {
+      case "string" => Field(field.column, Some("String"), field.name, Some(null), field.parent,
+        field.storeType, field.schemaOrdinal
+      )
+      case "smallint" => Field(field.column, Some("SmallInt"), field.name, Some(null),
+        field.parent, field.storeType, field.schemaOrdinal
+      )
+      case "integer" | "int" => Field(field.column, Some("Integer"), field.name, Some(null),
+        field.parent, field.storeType, field.schemaOrdinal
+      )
+      case "long" => Field(field.column, Some("Long"), field.name, Some(null), field.parent,
+        field.storeType, field.schemaOrdinal
+      )
+      case "double" => Field(field.column, Some("Double"), field.name, Some(null), field.parent,
+        field.storeType, field.schemaOrdinal
+      )
+      case "timestamp" => Field(field.column, Some("Timestamp"), field.name, Some(null),
+        field.parent, field.storeType, field.schemaOrdinal
+      )
+      case "numeric" => Field(field.column, Some("Numeric"), field.name, Some(null), field.parent,
+        field.storeType, field.schemaOrdinal
+      )
+      case "array" => Field(field.column, Some("Array"), field.name,
+        field.children.map(f => f.map(normalizeType(_))),
+        field.parent, field.storeType, field.schemaOrdinal
+      )
+      case "struct" => Field(field.column, Some("Struct"), field.name,
+        field.children.map(f => f.map(normalizeType(_))),
+        field.parent, field.storeType, field.schemaOrdinal
+      )
+      case "bigint" => Field(field.column, Some("BigInt"), field.name, Some(null), field.parent,
+        field.storeType, field.schemaOrdinal
+      )
+      case "decimal" => Field(field.column, Some("Decimal"), field.name, Some(null), field.parent,
+        field.storeType, field.schemaOrdinal, field.precision, field.scale
+      )
+      // checking if the nested data type contains the child type as decimal(10,0),
+      // if it is present then extracting the precision and scale. resetting the data type
+      // with Decimal.
+      case _ if (dataType.startsWith("decimal")) =>
+        val (precision, scale) = getScaleAndPrecision(dataType)
+        Field(field.column,
+          Some("Decimal"),
+          field.name,
+          Some(null),
+          field.parent,
+          field.storeType, field.schemaOrdinal, precision,
+          scale
+        )
+      case _ =>
+        field
+    }
+  }
+
+  private def addParent(field: Field): Field = {
+    field.dataType.getOrElse("NIL") match {
+      case "Array" => Field(field.column, Some("Array"), field.name,
+        field.children.map(f => f.map(appendParentForEachChild(_, field.column))), field.parent,
+        field.storeType, field.schemaOrdinal)
+      case "Struct" => Field(field.column, Some("Struct"), field.name,
+        field.children.map(f => f.map(appendParentForEachChild(_, field.column))), field.parent,
+        field.storeType, field.schemaOrdinal)
+      case _ => field
+    }
+  }
+
+  private def appendParentForEachChild(field: Field, parentName: String): Field = {
+    field.dataType.getOrElse("NIL") match {
+      case "String" => Field(parentName + "." + field.column, Some("String"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "SmallInt" => Field(parentName + "." + field.column, Some("SmallInt"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Integer" => Field(parentName + "." + field.column, Some("Integer"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Long" => Field(parentName + "." + field.column, Some("Long"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Double" => Field(parentName + "." + field.column, Some("Double"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Timestamp" => Field(parentName + "." + field.column, Some("Timestamp"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Numeric" => Field(parentName + "." + field.column, Some("Numeric"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Array" => Field(parentName + "." + field.column, Some("Array"),
+        Some(parentName + "." + field.name.getOrElse(None)),
+        field.children
+          .map(f => f.map(appendParentForEachChild(_, parentName + "." + field.column))),
+        parentName)
+      case "Struct" => Field(parentName + "." + field.column, Some("Struct"),
+        Some(parentName + "." + field.name.getOrElse(None)),
+        field.children
+          .map(f => f.map(appendParentForEachChild(_, parentName + "." + field.column))),
+        parentName)
+      case "BigInt" => Field(parentName + "." + field.column, Some("BigInt"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Decimal" => Field(parentName + "." + field.column, Some("Decimal"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName,
+        field.storeType, field.schemaOrdinal, field.precision, field.scale)
+      case _ => field
+    }
+  }
+
   protected lazy val showLoads: Parser[LogicalPlan] =
     SHOW ~> SEGMENTS ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident ~
     (LIMIT ~> numericLit).? <~
@@ -386,6 +1152,13 @@ class CarbonSqlParser() extends CarbonDDLSqlParser {
         ShowLoadsCommand(databaseName, tableName.toLowerCase(), limit)
     }
 
+  protected lazy val segmentId: Parser[String] =
+    numericLit ^^ { u => u } |
+    elem("decimal", p => {
+      p.getClass.getSimpleName.equals("FloatLit") ||
+      p.getClass.getSimpleName.equals("DecimalLit")
+    }) ^^ (_.chars)
+
   protected lazy val deleteLoadsByID: Parser[LogicalPlan] =
     DELETE ~> SEGMENT ~> repsep(segmentId, ",") ~ (FROM ~> TABLE ~>
                                                    (ident <~ ".").? ~ ident) <~
@@ -421,4 +1194,133 @@ class CarbonSqlParser() extends CarbonDDLSqlParser {
         }
     }
 
+  protected lazy val deleteRecords: Parser[LogicalPlan] =
+    (DELETE ~> FROM ~> table) ~ (WHERE ~> restInput).? <~  opt(";") ^^ {
+      case table ~ condition =>
+        val tableName = getTableName(table.tableIdentifier)
+        val alias = table.alias.getOrElse("")
+        val stmt = condition match {
+          case Some(cond) =>
+            "select tupleId from " + tableName  + " " + alias + " where " + cond
+          case _ =>
+            "select tupleId from " + tableName
+        }
+        DeleteRecords(stmt, table)
+    }
+
+  protected lazy val updateTable: Parser[LogicalPlan] =
+    UPDATE ~> table ~
+      (SET ~> "(" ~>  repsep(element, ",") <~ ")") ~
+      ( "=" ~> restInput ) <~ opt(";")  ^^ {
+      case  tab~ columns ~ rest =>
+        val (sel, where ) = splitQuery(rest)
+        val (selectStmt, relation) =
+          if (!sel.toLowerCase.startsWith("select ")) {
+            if (sel.trim.isEmpty) {
+              sys.error("At least one source column has to be specified ")
+            }
+            // only list of expression are given, need to convert that list of expressions into
+            // select statement on destination table
+            val relation = tab match {
+              case r@UnresolvedRelation(tableIdentifier, alias) =>
+                updateRelation(r, tableIdentifier, alias)
+              case _ => tab
+            }
+            ("select " + sel + " from " + getTableName(relation.tableIdentifier) + " " +
+              relation.alias.get, relation)
+          } else {
+            (sel, updateRelation(tab, tab.tableIdentifier, tab.alias))
+          }
+        UpdateTable(relation, columns, selectStmt, where)
+    }
+
+  private def splitQuery(query: String): (String, String) = {
+    val stack = scala.collection.mutable.Stack[Char]()
+    var foundSingleQuotes = false
+    var foundDoubleQuotes = false
+    var foundEscapeChar = false
+    var ignoreChar = false
+    var stop = false
+    var bracketCount = 0
+    val (selectStatement, where) = query.span {
+      ch => {
+        if (stop) {
+          false
+        } else {
+          ignoreChar = false
+          if (foundEscapeChar && (ch == '\'' || ch == '\"' || ch == '\\')) {
+            foundEscapeChar = false
+            ignoreChar = true
+          }
+          // If escaped single or double quotes found, no need to consider
+          if (!ignoreChar) {
+            if (ch == '\\') {
+              foundEscapeChar = true
+            } else if (ch == '\'') {
+              foundSingleQuotes = !foundSingleQuotes
+            } else if (ch == '\"') {
+              foundDoubleQuotes = !foundDoubleQuotes
+            }
+            else if (ch == '(' &&  !foundSingleQuotes && !foundDoubleQuotes) {
+              bracketCount = bracketCount + 1
+              stack.push(ch)
+            } else if (ch == ')' && !foundSingleQuotes && !foundDoubleQuotes) {
+              bracketCount = bracketCount + 1
+              stack.pop()
+              if (0 == stack.size) {
+                stop = true
+              }
+            }
+          }
+          true
+        }
+      }
+    }
+    if (bracketCount == 0 || bracketCount % 2 != 0) {
+      sys.error("Parsing error, missing bracket ")
+    }
+    val select = selectStatement.trim
+    (select.substring(1, select.length - 1).trim -> where.trim)
+  }
+
+
+  protected lazy val table: Parser[UnresolvedRelation] = {
+    rep1sep(attributeName, ".")  ~  opt(ident)^^ {
+      case tableIdent ~ alias => UnresolvedRelation(tableIdent, alias)
+    }
+  }
+
+  protected lazy val attributeName: Parser[String] = acceptMatch("attribute name", {
+    case lexical.Identifier(str) => str.toLowerCase
+    case lexical.Keyword(str) if !lexical.delimiters.contains(str) => str.toLowerCase
+  })
+
+  private def updateRelation(
+                              r: UnresolvedRelation,
+                              tableIdentifier: Seq[String],
+                              alias: Option[String]): UnresolvedRelation = {
+    alias match {
+      case Some(_) => r
+      case _ =>
+        val tableAlias = tableIdentifier match {
+          case Seq(dbName, tableName) => Some(tableName)
+          case Seq(tableName) => Some(tableName)
+        }
+        UnresolvedRelation(tableIdentifier, tableAlias)
+    }
+  }
+
+  private def getTableName(tableIdentifier: Seq[String]): String = {
+    if (tableIdentifier.size > 1) {
+      tableIdentifier(0) + "." + tableIdentifier(1)
+    } else {
+      tableIdentifier(0)
+    }
+  }
+
+  protected lazy val element: Parser[String] =
+    (ident <~ ".").? ~ ident  ^^ {
+      case table ~ column => column.toLowerCase
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/002279ec/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
index 0d996b7..a9b3afb 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
@@ -21,37 +21,36 @@ import java.io.IOException
 import java.text.SimpleDateFormat
 import java.util
 
-import org.apache.carbondata.common.iudprocessor.iuddata.RowCountDetailsVO
-import org.apache.carbondata.core.datastorage.store.impl.FileFactory
-import org.apache.carbondata.core.update.{CarbonUpdateUtil, DeleteDeltaBlockDetails, SegmentUpdateDetails, TupleIdEnum}
-import org.apache.carbondata.core.updatestatus.{SegmentStatusManager, SegmentUpdateStatusManager}
-import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
-import org.apache.carbondata.processing.exception.MultipleMatchingException
-import org.apache.carbondata.spark.util.QueryPlanUtil
-import org.apache.spark.sql.catalyst.TableIdentifier
-
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ListBuffer
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
 import org.apache.spark.sql.execution.RunnableCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.util.FileUtils
 
+import org.apache.carbondata.common.iudprocessor.iuddata.RowCountDetailsVO
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier
 import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.carbon.path.{CarbonTablePath, CarbonStorePath}
+import org.apache.carbondata.core.carbon.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory
+import org.apache.carbondata.core.update.{CarbonUpdateUtil, DeleteDeltaBlockDetails, SegmentUpdateDetails, TupleIdEnum}
+import org.apache.carbondata.core.updatestatus.{SegmentStatusManager, SegmentUpdateStatusManager}
 import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
 import org.apache.carbondata.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
+import org.apache.carbondata.processing.exception.MultipleMatchingException
 import org.apache.carbondata.spark.load.{CarbonLoaderUtil, FailureCauses}
+import org.apache.carbondata.spark.merger.{CarbonDataMergerUtil, CarbonDataMergerUtilResult, CompactionType}
 import org.apache.carbondata.spark.merger.CarbonDataMergerUtil._
-import org.apache.carbondata.spark.merger.{CarbonDataMergerUtil, CompactionType, CarbonDataMergerUtilResult}
 import org.apache.carbondata.spark.DeleteDelataResultImpl
+import org.apache.carbondata.spark.util.QueryPlanUtil
 
 
 /**
@@ -517,7 +516,8 @@ object deleteExecution {
       .getCarbonTablePath(storeLocation,
         absoluteTableIdentifier.getCarbonTableIdentifier())
     var tableUpdateStatusPath = tablePath.getTableUpdateStatusFilePath
-    val totalSegments = SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath).length
+    val totalSegments =
+      SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath).length
     var factPath = tablePath.getFactDir
 
     var carbonTable = relation.tableMeta.carbonTable
@@ -765,8 +765,13 @@ object deleteExecution {
 
 object UpdateExecution {
 
-  def performUpdate(dataFrame: DataFrame, tableIdentifier: Seq[String], plan: LogicalPlan,
-                    sqlContext: SQLContext, currentTime: Long, executorErrors: ExecutionErrors): Unit = {
+  def performUpdate(
+         dataFrame: DataFrame,
+         tableIdentifier: Seq[String],
+         plan: LogicalPlan,
+         sqlContext: SQLContext,
+         currentTime: Long,
+         executorErrors: ExecutionErrors): Unit = {
 
     def isDestinationRelation(relation: CarbonDatasourceRelation): Boolean = {
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/002279ec/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 0904bbb..5706a7f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -18,6 +18,11 @@
 package org.apache.spark.sql.execution.command
 
 import java.io.File
+import java.text.SimpleDateFormat
+
+import org.apache.carbondata.core.updatestatus.SegmentStatusManager
+import org.apache.carbondata.locks.{LockUsage, CarbonLockFactory}
+
 import java.util.concurrent.Callable
 import java.util.concurrent.Executors
 import java.util.concurrent.ExecutorService
@@ -29,7 +34,7 @@ import scala.language.implicitConversions
 import org.apache.commons.lang3.StringUtils
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Cast, Literal}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.{RunnableCommand, SparkPlan}
 import org.apache.spark.sql.hive.CarbonMetastore
@@ -47,10 +52,9 @@ import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDime
 import org.apache.carbondata.core.carbon.path.CarbonStorePath
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory
-import org.apache.carbondata.core.dictionary.server.DictionaryServer
+import org.apache.carbondata.core.updatestatus.SegmentStatusManager
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.lcm.locks.{CarbonLockFactory, LockUsage}
-import org.apache.carbondata.lcm.status.SegmentStatusManager
+import org.apache.carbondata.locks.{CarbonLockFactory, LockUsage}
 import org.apache.carbondata.processing.constants.TableOptionConstant
 import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.carbondata.processing.model.CarbonLoadModel

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/002279ec/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index c224de8..07ed1d4 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -16,7 +16,6 @@
  */
 package org.apache.spark.sql.hive
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedFunction, UnresolvedRelation, UnresolvedStar}
@@ -27,6 +26,8 @@ import org.apache.spark.sql.catalyst.rules._
 import org.apache.spark.sql.execution.command.ProjectForDeleteCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
 /**
  * Insert into carbon table from other source
  */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/002279ec/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
index cf66bda..6c32c9d 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
@@ -20,8 +20,6 @@ package org.apache.spark.sql.hive
 import java.io._
 import java.util.UUID
 
-import org.apache.carbondata.locks.ZookeeperInit
-
 import scala.Array.canBuildFrom
 import scala.collection.mutable.ArrayBuffer
 import scala.language.implicitConversions
@@ -49,6 +47,7 @@ import org.apache.carbondata.core.reader.ThriftReader
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, CarbonUtil}
 import org.apache.carbondata.core.writer.ThriftWriter
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
+import org.apache.carbondata.locks.ZookeeperInit
 import org.apache.carbondata.spark.merger.TableMeta
 
 case class MetaData(var tablesMeta: ArrayBuffer[TableMeta])

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/002279ec/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
index 752b2b9..fea79a5 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
@@ -19,9 +19,6 @@ package org.apache.spark.sql.optimizer
 
 import java.util
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.spark.sql.execution.command.ProjectForUpdateCommand
-
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
@@ -31,12 +28,14 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.optimizer.Optimizer
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.command.ProjectForUpdateCommand
 import org.apache.spark.sql.execution.RunnableCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.types.{IntegerType, StringType}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatistic
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory
 import org.apache.carbondata.spark.{CarbonAliasDecoderRelation, CarbonFilters}
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/002279ec/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
index af97eb9..fdd314c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
@@ -22,16 +22,16 @@
  */
 package org.apache.carbondata.processing.model;
 
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.carbondata.core.carbon.CarbonDataLoadSchema;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.load.LoadMetadataDetails;
 import org.apache.carbondata.core.update.SegmentUpdateDetails;
 import org.apache.carbondata.core.updatestatus.SegmentUpdateStatusManager;
 
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.List;
-
 public class CarbonLoadModel implements Serializable {
   /**
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/002279ec/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java
index d9ff7e5..37ab533 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java
@@ -106,8 +106,9 @@ public class TimSort<K, Buffer> {
     assert c != null;
 
     int nRemaining  = hi - lo;
-    if (nRemaining < 2)
+    if (nRemaining < 2) {
       return;  // Arrays of size 0 and 1 are always sorted
+    }
 
     // If array is small, do a "mini-TimSort" with no merges
     if (nRemaining < MIN_MERGE) {
@@ -170,8 +171,9 @@ public class TimSort<K, Buffer> {
   @SuppressWarnings("fallthrough")
   private void binarySort(Buffer a, int lo, int hi, int start, Comparator<? super K> c) {
     assert lo <= start && start <= hi;
-    if (start == lo)
+    if (start == lo) {
       start++;
+    }
 
     K key0 = s.newKey();
     K key1 = s.newKey();
@@ -192,10 +194,12 @@ public class TimSort<K, Buffer> {
        */
       while (left < right) {
         int mid = (left + right) >>> 1;
-        if (c.compare(pivot, s.getKey(a, mid, key1)) < 0)
+        if (c.compare(pivot, s.getKey(a, mid, key1)) < 0) {
           right = mid;
-        else
+        }
+        else {
           left = mid + 1;
+        }
       }
       assert left == right;
 
@@ -209,9 +213,15 @@ public class TimSort<K, Buffer> {
       int n = start - left;  // The number of elements to move
       // Switch is just an optimization for arraycopy in default case
       switch (n) {
-        case 2:  s.copyElement(a, left + 1, a, left + 2);
-        case 1:  s.copyElement(a, left, a, left + 1);
+        case 2:  {
+          s.copyElement(a, left + 1, a, left + 2);
+          s.copyElement(a, left, a, left + 1);
           break;
+        }
+        case 1:  {
+          s.copyElement(a, left, a, left + 1);
+          break;
+        }
         default: s.copyRange(a, left, a, left + 1, n);
       }
       s.copyElement(pivotStore, 0, a, left);
@@ -246,20 +256,23 @@ public class TimSort<K, Buffer> {
   private int countRunAndMakeAscending(Buffer a, int lo, int hi, Comparator<? super K> c) {
     assert lo < hi;
     int runHi = lo + 1;
-    if (runHi == hi)
+    if (runHi == hi) {
       return 1;
+    }
 
     K key0 = s.newKey();
     K key1 = s.newKey();
 
     // Find end of run, and reverse range if descending
     if (c.compare(s.getKey(a, runHi++, key0), s.getKey(a, lo, key1)) < 0) { // Descending
-      while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) < 0)
+      while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) < 0) {
         runHi++;
+      }
       reverseRange(a, lo, runHi);
     } else {                              // Ascending
-      while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) >= 0)
+      while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) >= 0) {
         runHi++;
+      }
     }
 
     return runHi - lo;
@@ -429,10 +442,11 @@ public class TimSort<K, Buffer> {
     private void mergeCollapse() {
       while (stackSize > 1) {
         int n = stackSize - 2;
-        if ( (n >= 1 && runLen[n-1] <= runLen[n] + runLen[n+1])
-          || (n >= 2 && runLen[n-2] <= runLen[n] + runLen[n-1])) {
-          if (runLen[n - 1] < runLen[n + 1])
+        if ((n >= 1 && runLen[n-1] <= runLen[n] + runLen[n+1])
+            || (n >= 2 && runLen[n-2] <= runLen[n] + runLen[n-1])) {
+          if (runLen[n - 1] < runLen[n + 1]) {
             n--;
+          }
         } else if (runLen[n] > runLen[n + 1]) {
           break; // Invariant is established
         }
@@ -447,8 +461,9 @@ public class TimSort<K, Buffer> {
     private void mergeForceCollapse() {
       while (stackSize > 1) {
         int n = stackSize - 2;
-        if (n > 0 && runLen[n - 1] < runLen[n + 1])
+        if (n > 0 && runLen[n - 1] < runLen[n + 1]) {
           n--;
+        }
         mergeAt(n);
       }
     }
@@ -494,8 +509,9 @@ public class TimSort<K, Buffer> {
       assert k >= 0;
       base1 += k;
       len1 -= k;
-      if (len1 == 0)
+      if (len1 == 0) {
         return;
+      }
 
       /*
        * Find where the last element of run1 goes in run2. Subsequent elements
@@ -503,14 +519,17 @@ public class TimSort<K, Buffer> {
        */
       len2 = gallopLeft(s.getKey(a, base1 + len1 - 1, key0), a, base2, len2, len2 - 1, c);
       assert len2 >= 0;
-      if (len2 == 0)
+      if (len2 == 0) {
         return;
+      }
 
       // Merge remaining runs, using tmp array with min(len1, len2) elements
-      if (len1 <= len2)
+      if (len1 <= len2) {
         mergeLo(base1, len1, base2, len2);
-      else
+      }
+      else {
         mergeHi(base1, len1, base2, len2);
+      }
     }
 
     /**
@@ -543,11 +562,13 @@ public class TimSort<K, Buffer> {
         while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key0)) > 0) {
           lastOfs = ofs;
           ofs = (ofs << 1) + 1;
-          if (ofs <= 0)   // int overflow
+          if (ofs <= 0) {  // int overflow
             ofs = maxOfs;
+          }
         }
-        if (ofs > maxOfs)
+        if (ofs > maxOfs) {
           ofs = maxOfs;
+        }
 
         // Make offsets relative to base
         lastOfs += hint;
@@ -558,11 +579,13 @@ public class TimSort<K, Buffer> {
         while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key0)) <= 0) {
           lastOfs = ofs;
           ofs = (ofs << 1) + 1;
-          if (ofs <= 0)   // int overflow
+          if (ofs <= 0) {  // int overflow
             ofs = maxOfs;
+          }
         }
-        if (ofs > maxOfs)
+        if (ofs > maxOfs) {
           ofs = maxOfs;
+        }
 
         // Make offsets relative to base
         int tmp = lastOfs;
@@ -580,10 +603,12 @@ public class TimSort<K, Buffer> {
       while (lastOfs < ofs) {
         int m = lastOfs + ((ofs - lastOfs) >>> 1);
 
-        if (c.compare(key, s.getKey(a, base + m, key0)) > 0)
+        if (c.compare(key, s.getKey(a, base + m, key0)) > 0) {
           lastOfs = m + 1;  // a[base + m] < key
-        else
+        }
+        else {
           ofs = m;          // key <= a[base + m]
+        }
       }
       assert lastOfs == ofs;    // so a[base + ofs - 1] < key <= a[base + ofs]
       return ofs;
@@ -615,11 +640,13 @@ public class TimSort<K, Buffer> {
         while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key1)) < 0) {
           lastOfs = ofs;
           ofs = (ofs << 1) + 1;
-          if (ofs <= 0)   // int overflow
+          if (ofs <= 0) {  // int overflow
             ofs = maxOfs;
+          }
         }
-        if (ofs > maxOfs)
+        if (ofs > maxOfs) {
           ofs = maxOfs;
+        }
 
         // Make offsets relative to b
         int tmp = lastOfs;
@@ -631,11 +658,13 @@ public class TimSort<K, Buffer> {
         while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key1)) >= 0) {
           lastOfs = ofs;
           ofs = (ofs << 1) + 1;
-          if (ofs <= 0)   // int overflow
+          if (ofs <= 0) {  // int overflow
             ofs = maxOfs;
+          }
         }
-        if (ofs > maxOfs)
+        if (ofs > maxOfs) {
           ofs = maxOfs;
+        }
 
         // Make offsets relative to b
         lastOfs += hint;
@@ -652,10 +681,12 @@ public class TimSort<K, Buffer> {
       while (lastOfs < ofs) {
         int m = lastOfs + ((ofs - lastOfs) >>> 1);
 
-        if (c.compare(key, s.getKey(a, base + m, key1)) < 0)
+        if (c.compare(key, s.getKey(a, base + m, key1)) < 0) {
           ofs = m;          // key < a[b + m]
-        else
+        }
+        else {
           lastOfs = m + 1;  // a[b + m] <= key
+        }
       }
       assert lastOfs == ofs;    // so a[b + ofs - 1] <= key < a[b + ofs]
       return ofs;
@@ -721,14 +752,16 @@ public class TimSort<K, Buffer> {
             s.copyElement(a, cursor2++, a, dest++);
             count2++;
             count1 = 0;
-            if (--len2 == 0)
+            if (--len2 == 0) {
               break outer;
+            }
           } else {
             s.copyElement(tmp, cursor1++, a, dest++);
             count1++;
             count2 = 0;
-            if (--len1 == 1)
+            if (--len1 == 1) {
               break outer;
+            }
           }
         } while ((count1 | count2) < minGallop);
 
@@ -745,12 +778,14 @@ public class TimSort<K, Buffer> {
             dest += count1;
             cursor1 += count1;
             len1 -= count1;
-            if (len1 <= 1) // len1 == 1 || len1 == 0
+            if (len1 <= 1) { // len1 == 1 || len1 == 0
               break outer;
+            }
           }
           s.copyElement(a, cursor2++, a, dest++);
-          if (--len2 == 0)
+          if (--len2 == 0) {
             break outer;
+          }
 
           count2 = gallopLeft(s.getKey(tmp, cursor1, key0), a, cursor2, len2, 0, c);
           if (count2 != 0) {
@@ -758,16 +793,19 @@ public class TimSort<K, Buffer> {
             dest += count2;
             cursor2 += count2;
             len2 -= count2;
-            if (len2 == 0)
+            if (len2 == 0) {
               break outer;
+            }
           }
           s.copyElement(tmp, cursor1++, a, dest++);
-          if (--len1 == 1)
+          if (--len1 == 1) {
             break outer;
+          }
           minGallop--;
         } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP);
-        if (minGallop < 0)
+        if (minGallop < 0) {
           minGallop = 0;
+        }
         minGallop += 2;  // Penalize for leaving gallop mode
       }  // End of "outer" loop
       this.minGallop = minGallop < 1 ? 1 : minGallop;  // Write back to field
@@ -843,14 +881,16 @@ public class TimSort<K, Buffer> {
             s.copyElement(a, cursor1--, a, dest--);
             count1++;
             count2 = 0;
-            if (--len1 == 0)
+            if (--len1 == 0) {
               break outer;
+            }
           } else {
             s.copyElement(tmp, cursor2--, a, dest--);
             count2++;
             count1 = 0;
-            if (--len2 == 1)
+            if (--len2 == 1) {
               break outer;
+            }
           }
         } while ((count1 | count2) < minGallop);
 
@@ -867,12 +907,14 @@ public class TimSort<K, Buffer> {
             cursor1 -= count1;
             len1 -= count1;
             s.copyRange(a, cursor1 + 1, a, dest + 1, count1);
-            if (len1 == 0)
+            if (len1 == 0) {
               break outer;
+            }
           }
           s.copyElement(tmp, cursor2--, a, dest--);
-          if (--len2 == 1)
+          if (--len2 == 1) {
             break outer;
+          }
 
           count2 = len2 - gallopLeft(s.getKey(a, cursor1, key0), tmp, 0, len2, len2 - 1, c);
           if (count2 != 0) {
@@ -880,16 +922,19 @@ public class TimSort<K, Buffer> {
             cursor2 -= count2;
             len2 -= count2;
             s.copyRange(tmp, cursor2 + 1, a, dest + 1, count2);
-            if (len2 <= 1)  // len2 == 1 || len2 == 0
+            if (len2 <= 1) { // len2 == 1 || len2 == 0
               break outer;
+            }
           }
           s.copyElement(a, cursor1--, a, dest--);
-          if (--len1 == 0)
+          if (--len1 == 0) {
             break outer;
+          }
           minGallop--;
         } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP);
-        if (minGallop < 0)
+        if (minGallop < 0) {
           minGallop = 0;
+        }
         minGallop += 2;  // Penalize for leaving gallop mode
       }  // End of "outer" loop
       this.minGallop = minGallop < 1 ? 1 : minGallop;  // Write back to field
@@ -929,10 +974,12 @@ public class TimSort<K, Buffer> {
         newSize |= newSize >> 16;
         newSize++;
 
-        if (newSize < 0) // Not bloody likely!
+        if (newSize < 0) { // Not bloody likely!
           newSize = minCapacity;
-        else
+        }
+        else {
           newSize = Math.min(newSize, aLength >>> 1);
+        }
 
         tmp = s.allocate(newSize);
         tmpLength = newSize;


Mime
View raw message