carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jack...@apache.org
Subject [10/14] incubator-carbondata git commit: remove some comments and code standard
Date Mon, 08 May 2017 09:48:22 GMT
remove some comments and code standard


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

Branch: refs/heads/12-dev
Commit: a14f92c9a26baecfcff9d115a461327837bc4648
Parents: 46ab714
Author: lionelcao <whucaolu@gmail.com>
Authored: Mon May 8 14:29:11 2017 +0800
Committer: lionelcao <whucaolu@gmail.com>
Committed: Mon May 8 14:29:11 2017 +0800

----------------------------------------------------------------------
 .../core/metadata/schema/PartitionInfo.java     |  19 ----
 .../carbondata/spark/util/CommonUtil.scala      |  15 ++-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala | 110 +++++++++----------
 .../spark/sql/parser/CarbonSparkSqlParser.scala |  19 ++--
 4 files changed, 72 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a14f92c9/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
index 6a7af9d..96d2eeb 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
@@ -23,37 +23,18 @@ import java.util.List;
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 
-/**
- * Partition Information of carbon partition table
- */
 public class PartitionInfo implements Serializable {
 
-  /**
-   * Partition columns
-   */
   private List<ColumnSchema> columnSchemaList;
 
   private PartitionType partitionType;
 
-  /**
-   * Range Partition definition
-   */
   private List<String> rangeInfo;
 
-  /**
-   * List Partition definition
-   */
   private List<List<String>> listInfo;
 
-  /**
-   * Hash Partition numbers
-   */
   private int hashNumber;
 
-  /**
-   * @param columnSchemaList
-   * @param partitionType
-   */
   public PartitionInfo(List<ColumnSchema> columnSchemaList, PartitionType partitionType)
{
     this.columnSchemaList = columnSchemaList;
     this.partitionType = partitionType;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a14f92c9/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 f40cc4e..196d1e4 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
@@ -26,7 +26,7 @@ import org.apache.commons.lang3.StringUtils
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.spark.SparkContext
-import org.apache.spark.sql.execution.command.{ColumnProperty, Field}
+import org.apache.spark.sql.execution.command.{ColumnProperty, Field, PartitionerField}
 import org.apache.spark.sql.types.StructField
 import org.apache.spark.util.FileUtils
 
@@ -152,21 +152,20 @@ object CommonUtil {
    * 4. If partition_type is Range, then range_info should be defined
    * 5. Only support single level partition for now
    * @param tableProperties
-   * @param partitionCols
-   * @return
+   * @param partitionerFields
+   * @return partition clause and definition in tblproperties are valid or not
    */
   def validatePartitionColumns(tableProperties: Map[String, String],
-      partitionCols: Seq[StructField]): Boolean = {
+      partitionerFields: Seq[PartitionerField]): Boolean = {
     var isValid: Boolean = true
     val partitionType = tableProperties.get(CarbonCommonConstants.PARTITION_TYPE)
     val hashNumber = tableProperties.get(CarbonCommonConstants.HASH_NUMBER)
     val rangeInfo = tableProperties.get(CarbonCommonConstants.RANGE_INFO)
     val listInfo = tableProperties.get(CarbonCommonConstants.LIST_INFO)
 
-    // partition column and partition_type should be both exist or not exist
-    if (partitionCols.isEmpty ^ partitionType.isEmpty) {
+    if (partitionType.isEmpty) {
       isValid = false
-    } else if (partitionCols.nonEmpty) {
+    } else {
       partitionType.get.toUpperCase() match {
         case "HASH" => if (!hashNumber.isDefined) isValid = false
         case "LIST" => if (!listInfo.isDefined) isValid = false
@@ -175,7 +174,7 @@ object CommonUtil {
         case _ => isValid = false
       }
       // only support one partition column for now
-      if (partitionCols.length > 1) isValid = false
+      if (partitionerFields.length > 1) isValid = false
     }
     isValid
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a14f92c9/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 2fbd003..954c453 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
@@ -262,11 +262,8 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser
{
     // get no inverted index columns from table properties.
     val noInvertedIdxCols = extractNoInvertedIndexColumns(fields, tableProperties)
     // get partitionInfo
-    var partitionInfo: Option[PartitionInfo] = None
+    val partitionInfo = getPartitionInfo(partitionCols, tableProperties)
 
-    if (!(partitionCols.length < 1)) {
-      partitionInfo = getPartitionInfo(partitionCols, tableProperties)
-    }
     // validate the tableBlockSize from table properties
     CommonUtil.validateTableBlockSize(tableProperties)
 
@@ -356,65 +353,68 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser
{
   }
 
   /**
-   * get partition info
    * @param partitionCols
    * @param tableProperties
    */
   protected def getPartitionInfo(partitionCols: Seq[PartitionerField],
       tableProperties: Map[String, String]): Option[PartitionInfo] = {
-    var partitionType: String = ""
-    var hashNumber = 0
-    var rangeInfo = List[String]()
-    var listInfo = ListBuffer[List[String]]()
-    var templist = ListBuffer[String]()
-    if (tableProperties.get(CarbonCommonConstants.PARTITION_TYPE).isDefined) {
-      partitionType = tableProperties.get(CarbonCommonConstants.PARTITION_TYPE).get
-    }
-    if (tableProperties.get(CarbonCommonConstants.HASH_NUMBER).isDefined) {
-      hashNumber = tableProperties.get(CarbonCommonConstants.HASH_NUMBER).get.toInt
-    }
-    if (tableProperties.get(CarbonCommonConstants.RANGE_INFO).isDefined) {
-      rangeInfo = tableProperties.get(CarbonCommonConstants.RANGE_INFO).get.split(",")
-        .map(_.trim()).toList
-    }
-    if (tableProperties.get(CarbonCommonConstants.LIST_INFO).isDefined) {
-      val arr = tableProperties.get(CarbonCommonConstants.LIST_INFO).get.split(",")
-        .map(_.trim())
-      val iter = arr.iterator
-      while(iter.hasNext) {
-        val value = iter.next()
-        if (value.startsWith("(")) {
-          templist += value.replace("(", "").trim()
-        } else if (value.endsWith(")")) {
-          templist += value.replace(")", "").trim()
-          listInfo += templist.toList
-          templist.clear()
-        } else {
-          templist += value
-          listInfo += templist.toList
-          templist.clear()
+    if (partitionCols.isEmpty) {
+      None
+    } else {
+      var partitionType: String = ""
+      var hashNumber = 0
+      var rangeInfo = List[String]()
+      var listInfo = ListBuffer[List[String]]()
+      var templist = ListBuffer[String]()
+      if (tableProperties.get(CarbonCommonConstants.PARTITION_TYPE).isDefined) {
+        partitionType = tableProperties.get(CarbonCommonConstants.PARTITION_TYPE).get
+      }
+      if (tableProperties.get(CarbonCommonConstants.HASH_NUMBER).isDefined) {
+        hashNumber = tableProperties.get(CarbonCommonConstants.HASH_NUMBER).get.toInt
+      }
+      if (tableProperties.get(CarbonCommonConstants.RANGE_INFO).isDefined) {
+        rangeInfo = tableProperties.get(CarbonCommonConstants.RANGE_INFO).get.split(",")
+          .map(_.trim()).toList
+      }
+      if (tableProperties.get(CarbonCommonConstants.LIST_INFO).isDefined) {
+        val arr = tableProperties.get(CarbonCommonConstants.LIST_INFO).get.split(",")
+          .map(_.trim())
+        val iter = arr.iterator
+        while (iter.hasNext) {
+          val value = iter.next()
+          if (value.startsWith("(")) {
+            templist += value.replace("(", "").trim()
+          } else if (value.endsWith(")")) {
+            templist += value.replace(")", "").trim()
+            listInfo += templist.toList
+            templist.clear()
+          } else {
+            templist += value
+            listInfo += templist.toList
+            templist.clear()
+          }
         }
       }
+      val cols : ArrayBuffer[ColumnSchema] = new ArrayBuffer[ColumnSchema]()
+      partitionCols.foreach(partition_col => {
+        val columnSchema = new ColumnSchema
+        columnSchema.setDataType(DataTypeConverterUtil.
+          convertToCarbonType(partition_col.dataType.get))
+        columnSchema.setColumnName(partition_col.partitionColumn)
+        cols += columnSchema
+      })
+
+      var partitionInfo : PartitionInfo = null
+      partitionType.toUpperCase() match {
+        case "HASH" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.HASH)
+          partitionInfo.setHashNumber(hashNumber)
+        case "RANGE" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.RANGE)
+          partitionInfo.setRangeInfo(rangeInfo.asJava)
+        case "LIST" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.LIST)
+          partitionInfo.setListInfo(listInfo.map(_.asJava).toList.asJava)
+      }
+      Some(partitionInfo)
     }
-    val cols : ArrayBuffer[ColumnSchema] = new ArrayBuffer[ColumnSchema]()
-    partitionCols.foreach(partition_col => {
-      val columnSchema = new ColumnSchema
-      columnSchema.setDataType(DataTypeConverterUtil.
-        convertToCarbonType(partition_col.dataType.get))
-      columnSchema.setColumnName(partition_col.partitionColumn)
-      cols += columnSchema
-    })
-
-    var partitionInfo : PartitionInfo = null
-    partitionType.toUpperCase() match {
-      case "HASH" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.HASH)
-                     partitionInfo.setHashNumber(hashNumber)
-      case "RANGE" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.RANGE)
-                      partitionInfo.setRangeInfo(rangeInfo.asJava)
-      case "LIST" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.LIST)
-                     partitionInfo.setListInfo(listInfo.map(_.asJava).toList.asJava)
-    }
-    Some(partitionInfo)
   }
 
   protected def extractColumnProperties(fields: Seq[Field], tableProperties: Map[String,
String]):

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a14f92c9/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index b2bd067..059c734 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -97,9 +97,9 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf)
{
       if (ctx.bucketSpec != null) {
         operationNotAllowed("CREATE TABLE ... CLUSTERED BY", ctx)
       }
-      val partitionCols = Option(ctx.partitionColumns).toSeq.flatMap(visitColTypeList)
-      val partitionFields = partitionCols.map(x =>
-                              PartitionerField(x.name, Some(x.dataType.toString), null))
+      val partitionerFields = Option(ctx.partitionColumns).toSeq.flatMap(visitColTypeList)
+        .map( structField =>
+            PartitionerField(structField.name, Some(structField.dataType.toString), null))
       val cols = Option(ctx.columns).toSeq.flatMap(visitColTypeList)
       val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues)
         .getOrElse(Map.empty)
@@ -115,14 +115,15 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf)
{
       }
 
       val tableProperties = mutable.Map[String, String]()
-      properties.foreach(f => tableProperties.put(f._1, f._2.toLowerCase))
+      properties.foreach{property => tableProperties.put(property._1, property._2.toLowerCase)}
 
       // validate partition clause
-      if (!CommonUtil.validatePartitionColumns(tableProperties, partitionCols)) {
-        throw new MalformedCarbonCommandException("Invalid partition definition")
-      } else if (partitionCols.nonEmpty) {
+      if (partitionerFields.nonEmpty) {
+        if (!CommonUtil.validatePartitionColumns(tableProperties, partitionerFields)) {
+          throw new MalformedCarbonCommandException("Invalid partition definition")
+        }
         // partition columns must be part of the schema
-        val badPartCols = partitionCols.map(_.name).toSet.intersect(colNames.toSet)
+        val badPartCols = partitionerFields.map(_.partitionColumn).toSet.intersect(colNames.toSet)
         if (badPartCols.isEmpty) {
           operationNotAllowed(s"Partition columns must be specified in the schema: " +
                               badPartCols.map("\"" + _ + "\"").mkString("[", ",", "]"), ctx)
@@ -184,7 +185,7 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf)
{
         convertDbNameToLowerCase(name.database),
         name.table.toLowerCase,
         fields,
-        partitionFields,
+        partitionerFields,
         tableProperties,
         bucketFields)
 


Mime
View raw message