You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2017/06/08 17:20:36 UTC

[1/2] carbondata git commit: fixed NullPointerException in partition column

Repository: carbondata
Updated Branches:
  refs/heads/master b754390e6 -> 2e4210135


fixed NullPointerException in partition column

fixed failing test cases

removed unused imports


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

Branch: refs/heads/master
Commit: b8346b47d3a3a56cd773ecd1fb2116f825b3cb66
Parents: b754390
Author: kunal642 <ku...@gmail.com>
Authored: Thu Jun 8 18:36:30 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Jun 8 22:48:48 2017 +0530

----------------------------------------------------------------------
 .../core/metadata/schema/table/CarbonTable.java | 20 +++++++++++++------
 .../execution/command/AlterTableCommands.scala  | 21 +++++++++++---------
 .../spark/sql/common/util/QueryTest.scala       |  5 +++--
 3 files changed, 29 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/b8346b47/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 9f8beb0..429b1df 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -169,13 +169,21 @@ public class CarbonTable implements Serializable {
     for (TableSchema aggTable : aggregateTableList) {
       this.aggregateTablesName.add(aggTable.getTableName());
       fillDimensionsAndMeasuresForTables(aggTable);
-      tableBucketMap.put(aggTable.getTableName(), aggTable.getBucketingInfo());
-      tablePartitionMap.put(aggTable.getTableName(), aggTable.getPartitionInfo());
+      if (aggTable.getBucketingInfo() != null) {
+        tableBucketMap.put(aggTable.getTableName(), aggTable.getBucketingInfo());
+      }
+      if (aggTable.getPartitionInfo() != null) {
+        tablePartitionMap.put(aggTable.getTableName(), aggTable.getPartitionInfo());
+      }
+    }
+    if (tableInfo.getFactTable().getBucketingInfo() != null) {
+      tableBucketMap.put(tableInfo.getFactTable().getTableName(),
+          tableInfo.getFactTable().getBucketingInfo());
+    }
+    if (tableInfo.getFactTable().getPartitionInfo() != null) {
+      tablePartitionMap.put(tableInfo.getFactTable().getTableName(),
+          tableInfo.getFactTable().getPartitionInfo());
     }
-    tableBucketMap.put(tableInfo.getFactTable().getTableName(),
-        tableInfo.getFactTable().getBucketingInfo());
-    tablePartitionMap.put(tableInfo.getFactTable().getTableName(),
-        tableInfo.getFactTable().getPartitionInfo());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b8346b47/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
index d6b4838..b08c113 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
@@ -282,15 +282,18 @@ private[sql] case class AlterTableDropColumns(
       carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
         .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
         .tableMeta.carbonTable
-      val columnNames = carbonTable.getPartitionInfo(tableName).getColumnSchemaList.asScala
-        .map(_.getColumnName)
-      // check each column existence in the table
-      val partitionColumns = alterTableDropColumnModel.columns.filter {
-        tableColumn => columnNames.contains(tableColumn)
-      }
-      if (partitionColumns.nonEmpty) {
-        throw new UnsupportedOperationException("Partition columns cannot be dropped: " +
-                                                s"$partitionColumns")
+      val partitionInfo = carbonTable.getPartitionInfo(tableName)
+      if (partitionInfo != null) {
+        val partitionColumnSchemaList = partitionInfo.getColumnSchemaList.asScala
+          .map(_.getColumnName)
+        // check each column existence in the table
+        val partitionColumns = alterTableDropColumnModel.columns.filter {
+          tableColumn => partitionColumnSchemaList.contains(tableColumn)
+        }
+        if (partitionColumns.nonEmpty) {
+          throw new UnsupportedOperationException("Partition columns cannot be dropped: " +
+                                                  s"$partitionColumns")
+        }
       }
       val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
       var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b8346b47/integration/spark2/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala b/integration/spark2/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
index c37ea1e..c56c0c3 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
@@ -23,7 +23,7 @@ import scala.collection.JavaConversions._
 
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.hive.CarbonSessionState
 import org.apache.spark.sql.test.TestQueryExecutor
 import org.apache.spark.sql.{DataFrame, Row}
 
@@ -40,7 +40,8 @@ class QueryTest extends PlanTest {
 
   val sqlContext = TestQueryExecutor.INSTANCE.sqlContext
 
-  val hiveClient = sqlContext.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val hiveClient = sqlContext.sparkSession.sessionState.asInstanceOf[CarbonSessionState]
+    .metadataHive
 
   val resourcesPath = TestQueryExecutor.resourcesPath
 


[2/2] carbondata git commit: [CARBONDATA-1144] Fixed NullPointerException in partition column This closes #1009

Posted by ra...@apache.org.
[CARBONDATA-1144] Fixed NullPointerException in partition column This closes #1009


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

Branch: refs/heads/master
Commit: 2e42101357d1f71b5d3bbd6681f80cf1b8cbb276
Parents: b754390 b8346b4
Author: ravipesala <ra...@gmail.com>
Authored: Thu Jun 8 22:50:16 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Jun 8 22:50:16 2017 +0530

----------------------------------------------------------------------
 .../core/metadata/schema/table/CarbonTable.java | 20 +++++++++++++------
 .../execution/command/AlterTableCommands.scala  | 21 +++++++++++---------
 .../spark/sql/common/util/QueryTest.scala       |  5 +++--
 3 files changed, 29 insertions(+), 17 deletions(-)
----------------------------------------------------------------------