carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jack...@apache.org
Subject [11/14] incubator-carbondata git commit: add some comments and change hash_number to Number_Of_Partitions
Date Mon, 08 May 2017 09:48:23 GMT
add some comments and change hash_number to Number_Of_Partitions


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

Branch: refs/heads/12-dev
Commit: a44d20d3fa78c8636ff6be4fdf2c9cc590ed957f
Parents: a14f92c
Author: lionelcao <whucaolu@gmail.com>
Authored: Mon May 8 16:13:21 2017 +0800
Committer: lionelcao <whucaolu@gmail.com>
Committed: Mon May 8 16:13:21 2017 +0800

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  2 +-
 .../core/metadata/schema/PartitionInfo.java     | 22 +++++++++++++++-----
 .../examples/CarbonPartitionExample.scala       |  2 +-
 format/src/main/thrift/schema.thrift            |  2 +-
 .../carbondata/spark/util/CommonUtil.scala      |  6 +++---
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  9 ++++----
 6 files changed, 28 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a44d20d3/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 55040a5..0023387 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -807,7 +807,7 @@ public final class CarbonCommonConstants {
   public static final String DICTIONARY_INCLUDE = "dictionary_include";
   public static final String SORT_COLUMNS = "sort_columns";
   public static final String PARTITION_TYPE = "partition_type";
-  public static final String HASH_NUMBER = "hash_number";
+  public static final String NUMBER_OF_PARTITIONS = "number_of_partitions";
   public static final String RANGE_INFO = "range_info";
   public static final String LIST_INFO = "list_info";
   public static final String COLUMN_PROPERTIES = "columnproperties";

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a44d20d3/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 96d2eeb..7c8edc3 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,17 +23,29 @@ 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 {
 
   private List<ColumnSchema> columnSchemaList;
 
   private PartitionType partitionType;
 
+  /**
+   * range infomation defined for range partition table
+   */
   private List<String> rangeInfo;
 
+  /**
+   * value list defined for list partition table
+   */
   private List<List<String>> listInfo;
 
-  private int hashNumber;
+  /**
+   * hash partition numbers
+   */
+  private int numberOfPartitions;
 
   public PartitionInfo(List<ColumnSchema> columnSchemaList, PartitionType partitionType)
{
     this.columnSchemaList = columnSchemaList;
@@ -48,12 +60,12 @@ public class PartitionInfo implements Serializable {
     return partitionType;
   }
 
-  public void setHashNumber(int hashNumber) {
-    this.hashNumber = hashNumber;
+  public void setNumberOfPartitions(int numberOfPartitions) {
+    this.numberOfPartitions = numberOfPartitions;
   }
 
-  public int getHashNumber() {
-    return hashNumber;
+  public int getNumberOfPartitions() {
+    return numberOfPartitions;
   }
 
   public void setRangeInfo(List<String> rangeInfo) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a44d20d3/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
index 72ccef8..93ade9c 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
@@ -95,7 +95,7 @@ object CarbonPartitionExample {
                 | )
                 | PARTITIONED BY (vin String)
                 | STORED BY 'carbondata'
-                | TBLPROPERTIES('PARTITION_TYPE'='HASH','HASH_NUMBER'='5')
+                | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUMBER_OF_PARTITIONS'='5')
                 """.stripMargin)
 
     // list partition

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a44d20d3/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 695333c..779c6e7 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -132,7 +132,7 @@ struct SchemaEvolution{
 struct PartitionInfo{
     1: required list<ColumnSchema> partition_columns;
     2: required PartitionType partition_type;
-    3: optional i32 hash_number;  // number of partitions defined in hash partition table
+    3: optional i32 number_of_partitions;  // number of partitions defined in hash partition
table
     4: optional list<list<string>> list_info; // value list of list partition
table
     5: optional list<string> range_info;  // range value list of range partition table
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a44d20d3/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 196d1e4..9e3e42a 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
@@ -147,7 +147,7 @@ object CommonUtil {
 
   /**
    * 1. If partitioned by clause exists, then partition_type should be defined
-   * 2. If partition_type is Hash, then hash_number should be defined
+   * 2. If partition_type is Hash, then number_of_partitions should be defined
    * 3. If partition_type is List, then list_info should be defined
    * 4. If partition_type is Range, then range_info should be defined
    * 5. Only support single level partition for now
@@ -159,7 +159,7 @@ object CommonUtil {
       partitionerFields: Seq[PartitionerField]): Boolean = {
     var isValid: Boolean = true
     val partitionType = tableProperties.get(CarbonCommonConstants.PARTITION_TYPE)
-    val hashNumber = tableProperties.get(CarbonCommonConstants.HASH_NUMBER)
+    val numberOfPartitions = tableProperties.get(CarbonCommonConstants.NUMBER_OF_PARTITIONS)
     val rangeInfo = tableProperties.get(CarbonCommonConstants.RANGE_INFO)
     val listInfo = tableProperties.get(CarbonCommonConstants.LIST_INFO)
 
@@ -167,7 +167,7 @@ object CommonUtil {
       isValid = false
     } else {
       partitionType.get.toUpperCase() match {
-        case "HASH" => if (!hashNumber.isDefined) isValid = false
+        case "HASH" => if (!numberOfPartitions.isDefined) isValid = false
         case "LIST" => if (!listInfo.isDefined) isValid = false
         case "RANGE" => if (!rangeInfo.isDefined) isValid = false
         case "RANGE_INTERVAL" => isValid = false

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a44d20d3/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 954c453..968f60b 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
@@ -362,15 +362,16 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser
{
       None
     } else {
       var partitionType: String = ""
-      var hashNumber = 0
+      var numberOfPartitions = 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.NUMBER_OF_PARTITIONS).isDefined) {
+        numberOfPartitions = tableProperties.get(CarbonCommonConstants.NUMBER_OF_PARTITIONS).get
+          .toInt
       }
       if (tableProperties.get(CarbonCommonConstants.RANGE_INFO).isDefined) {
         rangeInfo = tableProperties.get(CarbonCommonConstants.RANGE_INFO).get.split(",")
@@ -407,7 +408,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser
{
       var partitionInfo : PartitionInfo = null
       partitionType.toUpperCase() match {
         case "HASH" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.HASH)
-          partitionInfo.setHashNumber(hashNumber)
+          partitionInfo.setNumberOfPartitions(numberOfPartitions)
         case "RANGE" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.RANGE)
           partitionInfo.setRangeInfo(rangeInfo.asJava)
         case "LIST" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.LIST)


Mime
View raw message