You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2017/04/28 05:09:08 UTC

[1/2] incubator-carbondata git commit: insertInto read from OneRowRelation

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 61d3f3c2e -> 58be73fcd


insertInto read from OneRowRelation

fix comments


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

Branch: refs/heads/master
Commit: 2a4ab4b17c0ba24e98707589514d9887714c7685
Parents: 61d3f3c
Author: QiangCai <qi...@qq.com>
Authored: Thu Apr 13 18:39:12 2017 +0530
Committer: jackylk <ja...@huawei.com>
Committed: Thu Apr 27 22:08:21 2017 -0700

----------------------------------------------------------------------
 .../spark/sql/hive/CarbonAnalysisRules.scala    | 17 ++++++++-
 .../InsertIntoCarbonTableTestCase.scala         | 40 ++++++++++++++++++++
 2 files changed, 56 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/2a4ab4b1/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index 45accac..6061e3e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.hive
 
 import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, NamedExpression}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
 import org.apache.spark.sql.execution.datasources.LogicalRelation
@@ -52,7 +53,21 @@ object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
         )
     }
     if (child.output.size >= relation.carbonRelation.output.size) {
-      InsertIntoCarbonTable(relation, p.partition, p.child, p.overwrite, p.ifNotExists)
+      val newChildOutput = child.output.zipWithIndex.map { columnWithIndex =>
+        columnWithIndex._1 match {
+          case attr: Alias =>
+            Alias(attr.child, s"col${ columnWithIndex._2 }")(attr.exprId)
+          case attr: Attribute =>
+            Alias(attr, s"col${ columnWithIndex._2 }")(NamedExpression.newExprId)
+          case attr => attr
+        }
+      }
+      val newChild: LogicalPlan = if (newChildOutput == child.output) {
+        p.child
+      } else {
+        Project(newChildOutput, child)
+      }
+      InsertIntoCarbonTable(relation, p.partition, newChild, p.overwrite, p.ifNotExists)
     } else {
       sys.error("Cannot insert into target table because column number are different")
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/2a4ab4b1/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
new file mode 100644
index 0000000..0020568
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.spark.testsuite.allqueries
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll: Unit = {
+    sql("drop table if exists OneRowTable")
+  }
+
+  test("insert select one row") {
+    sql("create table OneRowTable(col1 string, col2 string, col3 int, col4 double) stored by 'carbondata'")
+    sql("insert into OneRowTable select '0.1', 'a.b', 1, 1.2")
+    checkAnswer(sql("select * from OneRowTable"), Seq(Row("0.1", "a.b", 1, 1.2)))
+  }
+
+  override def afterAll {
+    sql("drop table if exists OneRowTable")
+  }
+}


[2/2] incubator-carbondata git commit: [CARBONDATA-923] Fix issue of insertInto read from OneRowRelation This closes #795

Posted by ja...@apache.org.
[CARBONDATA-923] Fix issue of insertInto read from OneRowRelation This closes #795


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

Branch: refs/heads/master
Commit: 58be73fcdf51fa54891bff92594c0e4c73313751
Parents: 61d3f3c 2a4ab4b
Author: jackylk <ja...@huawei.com>
Authored: Thu Apr 27 22:08:47 2017 -0700
Committer: jackylk <ja...@huawei.com>
Committed: Thu Apr 27 22:08:47 2017 -0700

----------------------------------------------------------------------
 .../spark/sql/hive/CarbonAnalysisRules.scala    | 17 ++++++++-
 .../InsertIntoCarbonTableTestCase.scala         | 40 ++++++++++++++++++++
 2 files changed, 56 insertions(+), 1 deletion(-)
----------------------------------------------------------------------