You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/05/27 04:09:56 UTC

[GitHub] [carbondata] ShreelekhyaG opened a new pull request #3774: [WIP] Make geoID visible

ShreelekhyaG opened a new pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774


    ### Why is this PR needed?
    To make geohash column visible to the user
    
    ### What changes were proposed in this PR?
   Generated geohash column is included in the schema. Validation added to avoid alter and drop geohash column.
       
    ### Does this PR introduce any user interface change?
    - No
   
    ### Is any new testcase added?
    - Yes
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r456271810



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/CarbonSource.scala
##########
@@ -379,6 +380,17 @@ object CarbonSource {
     if (isCreatedByCarbonExtension(properties)) {
       // Table is created by SparkSession with CarbonExtension,
       // There is no TableInfo yet, so create it from CatalogTable
+      val columnSchema = updateTable.schema
+      val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+      // validate for spatial index column
+      columnSchema.foreach(x => {
+        if (spatialProperty.isDefined &&

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r460033679



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -170,11 +171,18 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       convertedStaticPartition)
     scanResultRdd = sparkSession.sessionState.executePlan(newLogicalPlan).toRdd
     if (logicalPartitionRelation != null) {
-      if (selectedColumnSchema.length != logicalPartitionRelation.output.length) {
+      val properties = table.getTableInfo.getFactTable.getTableProperties.asScala
+      val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+      var expectedOutput = logicalPartitionRelation.output
+      if (spatialProperty.isDefined && selectedColumnSchema.size + 1 == expectedOutput.length) {

Review comment:
       Yes, select *from table includes all columns including geoSpatial. I have added a testcase for that now. This change is when a user tries to insert with original schema. Like `sql(s"insert into $table1 select 1575428400000,116285807,40084087")` . As Spatial column is not given by the user and is internally generated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r458696449



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/geo/GeoTest.scala
##########
@@ -82,6 +81,33 @@ class GeoTest extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
       case None => assert(false)
     }
   }
+  test("test geo table drop spatial index column") {
+    createTable()
+    loadData()
+    val exception = intercept[MalformedCarbonCommandException](sql(s"alter table $table1 drop columns(mygeohash)"))
+    assert(exception.getMessage.contains(

Review comment:
       some more test case can be added with mixed case column names for other table property validation added, example range column, bucket column etc




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-666448308


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1827/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-636064979


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1369/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r458686578



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonParserUtil.scala
##########
@@ -677,6 +677,8 @@ object CarbonParserUtil {
         val errorMsg = "range_column not support multiple columns"
         throw new MalformedCarbonCommandException(errorMsg)
       }
+      CommonUtil.validateForSpatialTypeColumn(tableProperties, rangeColumn,

Review comment:
       Instead of checking property by property, once all the properties are filled, better to validate at one place  ?
   
   Also I see that for sortcolumns, column_metacache,no_inverted_index it is not handled




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-663586046


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1755/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r444648223



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -653,6 +653,21 @@ object CommonUtil {
     storeLocation
   }
 
+  def validateForSpatialTypeColumn(properties: Map[String, String],

Review comment:
       removed `validateSpatialIndexColumn()` from AlterTableUtil as validation to add column is no longer required, as spatial column is now part of schema.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [WIP] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-634249313






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA - 3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-635531765


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1368/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA - 3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-635396909


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1367/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667219322


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1844/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r437599439



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/geo/GeoTest.scala
##########
@@ -82,6 +81,45 @@ class GeoTest extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
       case None => assert(false)
     }
   }
+  test("test geo table drop spatial index column") {
+    createTable()
+    loadData()
+    val exception = intercept[MalformedCarbonCommandException](sql(s"alter table $table1 drop columns(mygeohash)"))
+    assert(exception.getMessage.contains(
+      s"Columns present in ${CarbonCommonConstants.SPATIAL_INDEX} table property cannot be altered."))
+  }
+
+  test("test geo table filter by geo spatial index column") {
+    createTable()
+    loadData()
+    checkAnswer(sql(s"select *from $table1 where mygeohash = '2196036'"),
+      Seq(Row(2196036,1575428400000L,116337069,39951887)))
+  }
+
+  test("test geo table create index on geohandler column") {
+    createTable()
+    loadData()
+    val exception = intercept[MalformedIndexCommandException](sql(
+      s"""
+         | CREATE INDEX bloom_index ON TABLE $table1 (mygeohash)
+         | AS 'bloomfilter'
+         | PROPERTIES('BLOOM_SIZE'='640000', 'BLOOM_FPP'='0.00001')
+      """.stripMargin))
+   assert(exception.getMessage.contains(
+      s"Spatial Index column is not supported, column 'mygeohash' is spatial column"))
+  }
+
+  test("test geo table custom properties on geohandler column") {
+    try {
+      createTable(table1,"'COLUMN_META_CACHE' = 'mygeohash',")

Review comment:
       Test framework supports interction of exceptions. Instead of this try catch and assert, please use intercept like above testcases.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r460034871



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
##########
@@ -70,12 +71,20 @@ private[sql] case class CarbonProjectForUpdateCommand(
     val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
     setAuditTable(carbonTable)
     setAuditInfo(Map("plan" -> plan.simpleString))
+    val properties = carbonTable.getTableInfo.getFactTable.getTableProperties.asScala
+    val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
     columns.foreach { col =>
       val dataType = carbonTable.getColumnByName(col).getColumnSchema.getDataType
       if (dataType.isComplexType) {
         throw new UnsupportedOperationException("Unsupported operation on Complex data type")
       }
-
+      if (spatialProperty.isDefined) {
+        if (col.contains(spatialProperty.get.trim)) {

Review comment:
       changed now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-648385173


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3197/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [WIP] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-634635351


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3076/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r444645924



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/geo/GeoTest.scala
##########
@@ -122,7 +161,8 @@ class GeoTest extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
     createTable(sourceTable)
     loadData(sourceTable)
     createTable(targetTable, "'SORT_SCOPE'='GLOBAL_SORT',")
-    sql(s"insert into  $targetTable select * from $sourceTable")
+    sql(s"insert into  $targetTable select timevalue, longitude," +

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-666112316


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667083404


   LGTM
   
   This PR can break backward compatibility in case of upgrade if anyone is using the geo spatial feature. But As I know it is an alpha feature. so breaking compatibility is OK.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-636504686


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1387/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA - 3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-635531975


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3089/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-654383362


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3308/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-636458697


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1381/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667792791


   Merging this. Thanks for your contribution!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r458697336



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
##########
@@ -209,6 +209,11 @@ private[sql] case class CarbonCreateSecondaryIndexCommand(
             .get
         }")
       }
+      val isSpatialColPresent = dims.find(x => x.getColumnSchema.isSpatialColumn)

Review comment:
       Please check , I think we need to remove isSpatialColumn from columnSchema. 
   we used that mainly for making  it invisible. Now as it is visible. It is just another plan column. Instead we can check if column name is in the table property or not.
   
   @VenuReddy2103 @ShreelekhyaG 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r460033679



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -170,11 +171,18 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       convertedStaticPartition)
     scanResultRdd = sparkSession.sessionState.executePlan(newLogicalPlan).toRdd
     if (logicalPartitionRelation != null) {
-      if (selectedColumnSchema.length != logicalPartitionRelation.output.length) {
+      val properties = table.getTableInfo.getFactTable.getTableProperties.asScala
+      val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+      var expectedOutput = logicalPartitionRelation.output
+      if (spatialProperty.isDefined && selectedColumnSchema.size + 1 == expectedOutput.length) {

Review comment:
       Yes, select *from table includes all columns including geoSpatial. I have added a testcase for that now. This change is when a user tries to insert with original schema. Like `sql(s"insert into $table1 select 1575428400000,116285807,40084087")`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on pull request #3774: [WIP] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-635226998


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r456275841



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -170,11 +171,18 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       convertedStaticPartition)
     scanResultRdd = sparkSession.sessionState.executePlan(newLogicalPlan).toRdd
     if (logicalPartitionRelation != null) {
-      if (selectedColumnSchema.length != logicalPartitionRelation.output.length) {
+      val properties = table.getTableInfo.getFactTable.getTableProperties.asScala

Review comment:
       In case of an insert with original schema had to remove to match with the input projection list.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r461692563



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
##########
@@ -210,6 +210,14 @@ private[sql] case class CarbonCreateSecondaryIndexCommand(
             .get
         }")
       }
+      val properties = carbonTable.getTableInfo.getFactTable.getTableProperties.asScala
+      val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+      if (spatialProperty.isDefined) {
+        if (dims.find(x => x.getColName.equalsIgnoreCase(spatialProperty.get.trim)).isDefined) {

Review comment:
       you would want to check for index column names.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r437700014



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -482,6 +490,7 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       null
     }
     var createOrderColumns = table.getCreateOrderColumn.asScala
+      .filterNot(_.getColumnSchema.isSpatialColumn)

Review comment:
       Please refer above comment. Also you would want to revisit `convertToNoDictionaryToBytes()` and `convertToNoDictionaryToBytesWithoutReArrange()` method in `InputProcessorStepWithNoConverterImpl` class. Without this PR, length of `data` and `dataFields` args of those method were not matching as we not including spatial column. so we had some change in them. With your PR it can be simplified. Please refer PR #3760.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667083485


   waiting for build to pass for the merge


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r458007728



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -482,6 +490,7 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       null
     }
     var createOrderColumns = table.getCreateOrderColumn.asScala
+      .filterNot(_.getColumnSchema.isSpatialColumn)

Review comment:
       even now `data` and `dataFields` args of the methods don't match when insert with original schema.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on pull request #3774: [CARBONDATA - 3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-635465726


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-666455308


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3566/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r458694598



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
##########
@@ -266,16 +266,24 @@ case class CarbonPreInsertionCasts(sparkSession: SparkSession) extends Rule[Logi
       relation: LogicalRelation,
       child: LogicalPlan): LogicalPlan = {
     val carbonDSRelation = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-    if (carbonDSRelation.carbonRelation.output.size > CarbonCommonConstants
+    val carbonTable = carbonDSRelation.carbonRelation.carbonTable
+    val properties = carbonTable.getTableInfo.getFactTable.getTableProperties.asScala
+    val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+    var expectedOutput = carbonDSRelation.carbonRelation.output
+    // have to remove geo column to support insert with original schema

Review comment:
       I think no need to remove this column from original schema as it is a visible column , same reason as I mentioned for `insertIntoCommand`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r461343500



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/geo/GeoTest.scala
##########
@@ -112,6 +238,23 @@ class GeoTest extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
       result)
   }
 
+  test("test insert into non-geo table select from geo table") {

Review comment:
       modified existing test case and added validation for the geo column.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
##########
@@ -209,6 +209,11 @@ private[sql] case class CarbonCreateSecondaryIndexCommand(
             .get
         }")
       }
+      val isSpatialColPresent = dims.find(x => x.getColumnSchema.isSpatialColumn)

Review comment:
       removed isSpatialColumn from schema.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
##########
@@ -228,8 +230,15 @@ class CarbonFileMetastore extends CarbonMetaStore {
             c.getClass.getName.equals("org.apache.spark.sql.catalyst.catalog.HiveTableRelation") ||
             c.getClass.getName.equals(
               "org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation")) =>
-        val catalogTable =
+        var catalogTable =
           CarbonReflectionUtils.getFieldOfCatalogTable("tableMeta", c).asInstanceOf[CatalogTable]
+        // remove spatial column from schema

Review comment:
       Here, catalogTable will have spatial column in schema which is used to build carbon table.  As spatial column is not supposed to be present in user-defined columns, removing it here. Later from tableproperties the column will be added in carbonTable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA - 3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-635397490


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3088/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-666343280


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r460688922



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/geo/GeoTest.scala
##########
@@ -112,6 +238,23 @@ class GeoTest extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
       result)
   }
 
+  test("test insert into non-geo table select from geo table") {

Review comment:
       please add a test case of insert into geo table, where insert rows will not have geo data. but select *  shows geo data




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] asfgit closed pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-648386101


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1471/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-665593991






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-664592745






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [WIP] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-635214295


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1362/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r458690774



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
##########
@@ -70,12 +71,20 @@ private[sql] case class CarbonProjectForUpdateCommand(
     val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
     setAuditTable(carbonTable)
     setAuditInfo(Map("plan" -> plan.simpleString))
+    val properties = carbonTable.getTableInfo.getFactTable.getTableProperties.asScala
+    val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
     columns.foreach { col =>
       val dataType = carbonTable.getColumnByName(col).getColumnSchema.getDataType
       if (dataType.isComplexType) {
         throw new UnsupportedOperationException("Unsupported operation on Complex data type")
       }
-
+      if (spatialProperty.isDefined) {
+        if (col.contains(spatialProperty.get.trim)) {

Review comment:
       why `contains` here ? it's a column name. suppose to be equalsIgnoreCase ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [WIP] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-635215816


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3083/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-665813658






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667138747


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] brijoobopanna commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
brijoobopanna commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-662238464


   retest this please
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-636458681


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3104/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r458689351



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -170,11 +171,18 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       convertedStaticPartition)
     scanResultRdd = sparkSession.sessionState.executePlan(newLogicalPlan).toRdd
     if (logicalPartitionRelation != null) {
-      if (selectedColumnSchema.length != logicalPartitionRelation.output.length) {
+      val properties = table.getTableInfo.getFactTable.getTableProperties.asScala
+      val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+      var expectedOutput = logicalPartitionRelation.output
+      if (spatialProperty.isDefined && selectedColumnSchema.size + 1 == expectedOutput.length) {

Review comment:
       why the changes in this function ?
   
   As user wanted to created geoSpatial column, we created an extra column, select * from table  should include all the columns.
   
   If the target table don't want geo column, user can specify projections.
   
   **we should not skip spatial column while doing insert into**
   
   @VenuReddy2103 , @ShreelekhyaG 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-666327149


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3561/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r437595830



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala
##########
@@ -568,8 +567,7 @@ object CommonLoadUtils {
         .getColumnSchemaList.size()))
         .asInstanceOf[Array[ColumnSchema]]
     } else {
-      colSchema = colSchema.filterNot(x => x.isInvisible || x.isComplexColumn ||
-                                           x.getSchemaOrdinal == -1)
+      colSchema = colSchema.filterNot(x => x.isInvisible || x.isComplexColumn)

Review comment:
       why remove getSchemaOrdinal check? we do not judge spatial column by schema ordinal as -1. And x.getSchemaOrdinal was not introduced with spatial support PR. Same applies to if case above.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667134434


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1842/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [WIP] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-634636348


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1355/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r460034270



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/geo/GeoTest.scala
##########
@@ -82,6 +81,33 @@ class GeoTest extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
       case None => assert(false)
     }
   }
+  test("test geo table drop spatial index column") {
+    createTable()
+    loadData()
+    val exception = intercept[MalformedCarbonCommandException](sql(s"alter table $table1 drop columns(mygeohash)"))
+    assert(exception.getMessage.contains(

Review comment:
       ok done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r437199601



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/geo/GeoTest.scala
##########
@@ -122,7 +161,8 @@ class GeoTest extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
     createTable(sourceTable)
     loadData(sourceTable)
     createTable(targetTable, "'SORT_SCOPE'='GLOBAL_SORT',")
-    sql(s"insert into  $targetTable select * from $sourceTable")
+    sql(s"insert into  $targetTable select timevalue, longitude," +

Review comment:
       why select * is changed to individual columns ? Probably because you get geospatial column as well when you give select *.  But, we need to handle this insert into select *.

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -653,6 +653,21 @@ object CommonUtil {
     storeLocation
   }
 
+  def validateForSpatialTypeColumn(properties: Map[String, String],

Review comment:
       `columnPropert` string is just to use that in exception message. we alreday have a similar method `validateSpatialIndexColumn()` for it. Suggest to reuse.

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -653,6 +653,21 @@ object CommonUtil {
     storeLocation
   }
 
+  def validateForSpatialTypeColumn(properties: Map[String, String],

Review comment:
       `columnPropert` string is just to use that in exception message. we alreday have a similar method `validateSpatialIndexColumn()` for it. Suggest to modify existing and reuse instead of complete new method for the same validation.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/CarbonSource.scala
##########
@@ -379,6 +380,17 @@ object CarbonSource {
     if (isCreatedByCarbonExtension(properties)) {
       // Table is created by SparkSession with CarbonExtension,
       // There is no TableInfo yet, so create it from CatalogTable
+      val columnSchema = updateTable.schema
+      val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+      // validate for spatial index column
+      columnSchema.foreach(x => {
+        if (spatialProperty.isDefined &&

Review comment:
       instead of `spatialProperty.isDefined`check inside the loop it should be outside. We can avoid unnecessary loop traversal if the property itself is not defined.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/CarbonSource.scala
##########
@@ -379,6 +380,17 @@ object CarbonSource {
     if (isCreatedByCarbonExtension(properties)) {
       // Table is created by SparkSession with CarbonExtension,
       // There is no TableInfo yet, so create it from CatalogTable
+      val columnSchema = updateTable.schema

Review comment:
       why moved this exception case alone here from `processSpatialIndexProperty` ? Probably we have missed something here. I think, we have many other invalid property exceptions in `processSpatialIndexProperty` and even for other properties in `prepareTableModel()`.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonParserUtil.scala
##########
@@ -213,8 +207,8 @@ object CarbonParserUtil {
 
     // Process spatial index property
     val indexFields = processSpatialIndexProperty(tableProperties, fields)
-    val allFields = fields ++ indexFields
 
+    val allFields = (fields ++ indexFields).distinct

Review comment:
       why distinct ? have already checked that indexFields are not in actual fields. right ? If required please add the comments here.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonParserUtil.scala
##########
@@ -213,8 +207,8 @@ object CarbonParserUtil {
 
     // Process spatial index property
     val indexFields = processSpatialIndexProperty(tableProperties, fields)
-    val allFields = fields ++ indexFields
 
+    val allFields = (fields ++ indexFields).distinct

Review comment:
       why distinct ? have already checked that indexFields are not in actual fields. right ? If required please add the comments here. Same applies for another instance below.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonParserUtil.scala
##########
@@ -337,7 +331,7 @@ object CarbonParserUtil {
 
     if (tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
       // validate the column_meta_cache option
-      val tableColumns = dims.view.filterNot(_.spatialIndex).map(x => x.name.get) ++
+      val tableColumns = dims.view.map(x => x.name.get) ++

Review comment:
       if we had not removed this filter, then you wouldn't require modification in `validateColumnMetaCacheFields` or `validateColumnMetaCacheOption` i guess.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonParserUtil.scala
##########
@@ -337,7 +331,7 @@ object CarbonParserUtil {
 
     if (tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
       // validate the column_meta_cache option
-      val tableColumns = dims.view.filterNot(_.spatialIndex).map(x => x.name.get) ++
+      val tableColumns = dims.view.map(x => x.name.get) ++

Review comment:
       if we had not removed this filter, then you wouldn't require modification in `validateColumnMetaCacheFields` `validateColumnMetaCacheOption`, `validateColumnMetaCacheAndCacheLevel` i guess.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -170,11 +171,18 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       convertedStaticPartition)
     scanResultRdd = sparkSession.sessionState.executePlan(newLogicalPlan).toRdd
     if (logicalPartitionRelation != null) {
-      if (selectedColumnSchema.length != logicalPartitionRelation.output.length) {
+      val properties = table.getTableInfo.getFactTable.getTableProperties.asScala

Review comment:
       In `getReArrangedIndexAndSelectedSchema()` above, we make `selectedColumnSchema` and return. In that method, we are excluding SpatialColumn column. That is the reason why the `selectedColumnSchema` & `logicalPartitionRelation.output` do not match here. I think, you need to fix there. then this change wouldn't be required anymore.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -170,11 +171,18 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       convertedStaticPartition)
     scanResultRdd = sparkSession.sessionState.executePlan(newLogicalPlan).toRdd
     if (logicalPartitionRelation != null) {
-      if (selectedColumnSchema.length != logicalPartitionRelation.output.length) {
+      val properties = table.getTableInfo.getFactTable.getTableProperties.asScala

Review comment:
       In `getReArrangedIndexAndSelectedSchema()` above, we make `selectedColumnSchema` and return. In that method, we are excluding SpatialColumn column. That is the reason why the `selectedColumnSchema` & `logicalPartitionRelation.output` do not match here. Probably you need to fix there. then this change wouldn't be required anymore.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoCommand.scala
##########
@@ -170,11 +171,18 @@ case class CarbonInsertIntoCommand(databaseNameOp: Option[String],
       convertedStaticPartition)
     scanResultRdd = sparkSession.sessionState.executePlan(newLogicalPlan).toRdd
     if (logicalPartitionRelation != null) {
-      if (selectedColumnSchema.length != logicalPartitionRelation.output.length) {
+      val properties = table.getTableInfo.getFactTable.getTableProperties.asScala

Review comment:
       In `getReArrangedIndexAndSelectedSchema()` above, we make `selectedColumnSchema` and return. In that method, we are excluding SpatialColumn column. That is the reason why the `selectedColumnSchema` & `logicalPartitionRelation.output` do not match here. Probably you need to fix there. then this change wouldn't be required anymore ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-636062526


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3090/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667219187


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3583/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-654179380






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-662284603


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1717/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r462128423



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
##########
@@ -210,6 +210,14 @@ private[sql] case class CarbonCreateSecondaryIndexCommand(
             .get
         }")
       }
+      val properties = carbonTable.getTableInfo.getFactTable.getTableProperties.asScala
+      val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+      if (spatialProperty.isDefined) {
+        if (dims.find(x => x.getColName.equalsIgnoreCase(spatialProperty.get.trim)).isDefined) {

Review comment:
       changed now

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
##########
@@ -266,16 +266,24 @@ case class CarbonPreInsertionCasts(sparkSession: SparkSession) extends Rule[Logi
       relation: LogicalRelation,
       child: LogicalPlan): LogicalPlan = {
     val carbonDSRelation = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-    if (carbonDSRelation.carbonRelation.output.size > CarbonCommonConstants
+    val carbonTable = carbonDSRelation.carbonRelation.carbonTable
+    val properties = carbonTable.getTableInfo.getFactTable.getTableProperties.asScala
+    val spatialProperty = properties.get(CarbonCommonConstants.SPATIAL_INDEX)
+    var expectedOutput = carbonDSRelation.carbonRelation.output
+    // have to remove geo column to support insert with original schema

Review comment:
       same reason as above. To support inset without geo column.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r458695151



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
##########
@@ -228,8 +230,15 @@ class CarbonFileMetastore extends CarbonMetaStore {
             c.getClass.getName.equals("org.apache.spark.sql.catalyst.catalog.HiveTableRelation") ||
             c.getClass.getName.equals(
               "org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation")) =>
-        val catalogTable =
+        var catalogTable =
           CarbonReflectionUtils.getFieldOfCatalogTable("tableMeta", c).asInstanceOf[CatalogTable]
+        // remove spatial column from schema

Review comment:
       same as above, we can register this column as it is visible column




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-663581874


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3497/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ydvpankaj99 commented on pull request #3774: [WIP] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ydvpankaj99 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-634578301


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-666333047


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1822/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-636504454


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3111/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r456272331



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonParserUtil.scala
##########
@@ -337,7 +331,7 @@ object CarbonParserUtil {
 
     if (tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
       // validate the column_meta_cache option
-      val tableColumns = dims.view.filterNot(_.spatialIndex).map(x => x.name.get) ++
+      val tableColumns = dims.view.map(x => x.name.get) ++

Review comment:
       yes, made changes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667129231


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3581/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-662284841


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3459/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r462751220



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
##########
@@ -228,8 +230,15 @@ class CarbonFileMetastore extends CarbonMetaStore {
             c.getClass.getName.equals("org.apache.spark.sql.catalyst.catalog.HiveTableRelation") ||
             c.getClass.getName.equals(
               "org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation")) =>
-        val catalogTable =
+        var catalogTable =
           CarbonReflectionUtils.getFieldOfCatalogTable("tableMeta", c).asInstanceOf[CatalogTable]
+        // remove spatial column from schema

Review comment:
       would have added this as comment !




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-654385185


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1570/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#discussion_r460035762



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonParserUtil.scala
##########
@@ -677,6 +677,8 @@ object CarbonParserUtil {
         val errorMsg = "range_column not support multiple columns"
         throw new MalformedCarbonCommandException(errorMsg)
       }
+      CommonUtil.validateForSpatialTypeColumn(tableProperties, rangeColumn,

Review comment:
       modified the method to validate all table properties in one place as suggested.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] VenuReddy2103 commented on pull request #3774: [CARBONDATA-3833] Make geoID visible

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on pull request #3774:
URL: https://github.com/apache/carbondata/pull/3774#issuecomment-667080685


   LGTM


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org