You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by mohammadshahidkhan <gi...@git.apache.org> on 2017/11/28 10:12:37 UTC

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

GitHub user mohammadshahidkhan opened a pull request:

    https://github.com/apache/carbondata/pull/1583

    [CARBONDATA-1822][Spark-Integration] Support DDL to register the CarbonData table from existing carbon table data

    
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
        Added new DDL command to register carbon tables using existing carbon table data/schema.
    
     - [ ] Any backward compatibility impacted?
        No
     - [ ] Document update required?
        Yes
     - [ ] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
           Yes 
           Added new test case to validate the register of normal table and aggregate table.
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mohammadshahidkhan/incubator-carbondata register_carbon

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/carbondata/pull/1583.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1583
    
----
commit a23aadeb75a68f9cc3fd5e18eb5e109aa66a1388
Author: mohammadshahidkhan <mo...@gmail.com>
Date:   2017-11-23T14:32:14Z

    [CARBONDATA-1822][Spark-Integration] Support DDL to register the CarbonData table from existing carbon table data

----


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1614/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    retest this please


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155045624
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala ---
    @@ -81,15 +90,13 @@ case class CarbonCreateTableCommand(
         val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tableIdentifier)
         if (createDSTable) {
           try {
    -        val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
    -        cm.dimCols.foreach(f => fields(f.schemaOrdinal) = f)
    -        cm.msrCols.foreach(f => fields(f.schemaOrdinal) = f)
    -
    -        sparkSession.sparkContext.setLocalProperty(EXECUTION_ID_KEY, null)
             val tablePath = tableIdentifier.getTablePath
    +        val carbonRelation = CarbonSparkUtil.createCarbonRelation(tableInfo, tablePath)
    +        val rawSchema = CarbonSparkUtil.getRawSchema(carbonRelation)
    --- End diff --
    
    While registering the table with hive we need rawSchema,
    earlier this we were getting from the column  Field


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1811/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1709/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1950/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1625/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154900579
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala ---
    @@ -523,4 +523,25 @@ class CarbonFileMetastore extends CarbonMetaStore {
         val tableMetadataFile = tablePath.getSchemaFilePath
         CarbonUtil.readSchemaFile(tableMetadataFile)
       }
    +
    +  /**
    +   * the method returns the Wrapper TableInfo
    +   *
    +   * @param absoluteTableIdentifier
    +   * @param sparkSession
    +   * @return
    +   */
    +  override def getWrapperTableInfo(absoluteTableIdentifier: AbsoluteTableIdentifier)
    --- End diff --
    
    Just name name as `getTableInfo`


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1991/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154983865
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -1044,48 +1045,48 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         dataType match {
           case "string" =>
             Field(field.column, Some("String"), field.name, Some(null), field.parent,
    -        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema
    -      )
    +        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema,
    +        field.columnComment)
           case "smallint" =>
             Field(field.column, Some("SmallInt"), field.name, Some(null),
               field.parent, field.storeType, field.schemaOrdinal,
    -          field.precision, field.scale, field.rawSchema)
    +          field.precision, field.scale, field.rawSchema, field.columnComment)
           case "integer" | "int" =>
             Field(field.column, Some("Integer"), field.name, Some(null),
               field.parent, field.storeType, field.schemaOrdinal,
    -          field.precision, field.scale, field.rawSchema)
    +          field.precision, field.scale, field.rawSchema, field.columnComment)
           case "long" => Field(field.column, Some("Long"), field.name, Some(null), field.parent,
    -        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema
    -      )
    +        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema,
    +        field.columnComment)
           case "double" => Field(field.column, Some("Double"), field.name, Some(null), field.parent,
    -        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema
    -      )
    +        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema,
    +        field.columnComment)
    --- End diff --
    
    Is this change for this PR? If not for this PR, please raise another PR for it


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2092/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1745/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1560/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1685/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154983469
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/events/RefreshTableEvents.scala ---
    @@ -0,0 +1,36 @@
    +/*
    + * 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.events
    +
    +import org.apache.spark.sql._
    +
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
    +
    +/**
    + * Class for handling operations before start of a load process.
    --- End diff --
    
    There already event to load process


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155269290
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    --- End diff --
    
    change to databaseNameOp, make it consistent with other command


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/439/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154986380
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala ---
    @@ -81,15 +90,13 @@ case class CarbonCreateTableCommand(
         val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tableIdentifier)
         if (createDSTable) {
           try {
    -        val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
    -        cm.dimCols.foreach(f => fields(f.schemaOrdinal) = f)
    -        cm.msrCols.foreach(f => fields(f.schemaOrdinal) = f)
    -
    -        sparkSession.sparkContext.setLocalProperty(EXECUTION_ID_KEY, null)
             val tablePath = tableIdentifier.getTablePath
    +        val carbonRelation = CarbonSparkUtil.createCarbonRelation(tableInfo, tablePath)
    +        val rawSchema = CarbonSparkUtil.getRawSchema(carbonRelation)
    --- End diff --
    
    Why this is needed


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2129/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2070/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2108/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154986226
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala ---
    @@ -29,25 +29,34 @@ import org.apache.carbondata.common.logging.LogServiceFactory
     import org.apache.carbondata.core.constants.CarbonCommonConstants
     import org.apache.carbondata.core.exception.InvalidConfigurationException
     import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
    -import org.apache.carbondata.core.metadata.schema.table.TableInfo
    +import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
     import org.apache.carbondata.core.util.CarbonUtil
     import org.apache.carbondata.events.{CreateTablePostExecutionEvent, CreateTablePreExecutionEvent, OperationContext, OperationListenerBus}
    +import org.apache.carbondata.spark.util.CarbonSparkUtil
     
     case class CarbonCreateTableCommand(
    -    cm: TableModel,
    +    tableInfo: TableInfo,
    --- End diff --
    
    I do not think you can create the TableInfo in parser, since it does not have the sparkSession, so the dbName maybe wrong. You can need to use `CarbonEnv.getDatabaseName` to get the dbName, it requires sparkSession


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1961/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/564/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154984440
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala ---
    @@ -320,13 +322,8 @@ class AlterTableColumnSchemaGenerator(
     // TODO: move this to carbon store API
     object TableNewProcessor {
       def apply(
    -      cm: TableModel,
    -      identifier: AbsoluteTableIdentifier): TableInfo = {
    -    new TableNewProcessor(
    -      cm,
    -      identifier.getDatabaseName,
    -      identifier.getTableName,
    -      identifier.getTablePath).process
    +      cm: TableModel): TableInfo = {
    --- End diff --
    
    Do not change this, otherwise tablePath will be lost


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154987030
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala ---
    @@ -142,6 +142,10 @@ trait CarbonMetaStore {
     
       def getTableFromMetadataCache(database: String, tableName: String): Option[CarbonTable]
     
    +  def getTableInfo(absoluteTableIdentifier: AbsoluteTableIdentifier)
    --- End diff --
    
    Why this is required? It is better not to add more interface


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1605/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2016/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1959/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155051361
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala ---
    @@ -47,6 +47,7 @@ import org.apache.carbondata.spark.util.DataTypeConverterUtil
     
     case class TableModel(
         ifNotExistsSet: Boolean,
    +    var databaseName: String,
    --- End diff --
    
    fixed


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1690/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1711/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/561/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155043480
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,213 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = CarbonEnv.getDatabaseName(dbName)(sparkSession)
    +    val databaseLocation = CarbonEnv.getDatabaseLocation(databaseName, sparkSession)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName
    --- End diff --
    
    fixed


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2128/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154276628
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,264 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{CreateTablePreExecutionEvent, OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +import org.apache.carbondata.spark.util.CommonUtil
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = GetDB.getDatabaseName(dbName, sparkSession)
    +    val databaseLocation = GetDB.getDatabaseLocation(databaseName, sparkSession,
    +      CarbonProperties.getStorePath)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
    +    var locks = List.empty[ICarbonLock]
    +    try {
    +      val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName
    +      val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
    +      locks = CommonUtil.acquireLock(locksToBeAcquired, absoluteTableIdentifier)
    +      // 2.1 check if the table already register with hive then ignore and continue with the next
    +      // schema
    +      if (!sparkSession.sessionState.catalog.listTables(databaseName)
    +        .exists(_.table.equalsIgnoreCase(tableName))) {
    +        val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
    --- End diff --
    
    moved to CarbonMetaStore.getWrapperTableInfo(absoluteTableIdentifier)


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154895427
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala ---
    @@ -876,5 +876,4 @@ object CommonUtil {
           }
         }
       }
    -
    --- End diff --
    
    remove unnecessary change


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1562/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154119145
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,264 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{CreateTablePreExecutionEvent, OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +import org.apache.carbondata.spark.util.CommonUtil
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = GetDB.getDatabaseName(dbName, sparkSession)
    +    val databaseLocation = GetDB.getDatabaseLocation(databaseName, sparkSession,
    +      CarbonProperties.getStorePath)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
    --- End diff --
    
    why locks are required to take here? we are not updating any files right, we are just updating to DB, I think concurrent scenarios it can take care internally.


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/carbondata/pull/1583


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1608/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154119662
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,264 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{CreateTablePreExecutionEvent, OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +import org.apache.carbondata.spark.util.CommonUtil
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = GetDB.getDatabaseName(dbName, sparkSession)
    +    val databaseLocation = GetDB.getDatabaseLocation(databaseName, sparkSession,
    +      CarbonProperties.getStorePath)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
    +    var locks = List.empty[ICarbonLock]
    +    try {
    +      val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName
    +      val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
    +      locks = CommonUtil.acquireLock(locksToBeAcquired, absoluteTableIdentifier)
    +      // 2.1 check if the table already register with hive then ignore and continue with the next
    +      // schema
    +      if (!sparkSession.sessionState.catalog.listTables(databaseName)
    +        .exists(_.table.equalsIgnoreCase(tableName))) {
    +        val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
    +        // check the existence of the schema file to know its a carbon table
    +        val schemaFilePath = carbonTablePath.getSchemaFilePath
    +        // if schema file does not exist then the table will either non carbon table or stale
    +        // carbon table
    +        if (FileFactory.isFileExist(schemaFilePath, FileFactory.getFileType(schemaFilePath))) {
    +          // read TableInfo
    +          val thriftTableInfo: org.apache.carbondata.format.TableInfo = metaStore
    +            .getThriftTableInfo(carbonTablePath)(sparkSession)
    +          val schemaConverter = new ThriftWrapperSchemaConverterImpl()
    +          val tableInfo = schemaConverter
    +            .fromExternalToWrapperTableInfo(thriftTableInfo,
    +              databaseName,
    +              tableName,
    +              absoluteTableIdentifier.getTablePath)
    +          // 2.2 register the table with the hive check if the table being registered has
    +          // aggregate table then do the below steps
    +          // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +          val dataMapSchemaList = tableInfo.getDataMapSchemaList
    +          if (null != dataMapSchemaList && dataMapSchemaList.size() != 0) {
    +            // validate all the aggregate tables are copied at the storeLocation
    +            val allExists = validateAllAggregateTablePresent(databaseName,
    +              dataMapSchemaList, databaseLocation)
    +            if (!allExists) {
    +              // fail the register operation
    +              val msg = s"Table registration with Database name [$databaseName] and Table name " +
    +                        s"[$tableName] failed. All the aggregate Tables for table [$tableName] is" +
    +                        s" not copied under database [$databaseName]"
    +              LOGGER.audit(msg)
    +              CarbonException.analysisException(msg)
    +            }
    +            // 2.2.1 Register the aggregate tables to hive
    +            registerAggregates(databaseName, dataMapSchemaList, databaseLocation)(sparkSession)
    +          }
    +          registerTableWithHive(databaseName, tableName, tableInfo)(sparkSession)
    +        } else {
    +          LOGGER.audit(
    +            s"Table registration with Database name [$databaseName] and Table name [$tableName] " +
    +            s"failed." +
    +            s"Table [$tableName] either non carbon table or stale carbon table under database " +
    +            s"[$databaseName]")
    +        }
    +      } else {
    +        LOGGER.audit(
    +          s"Table registration with Database name [$databaseName] and Table name [$tableName] " +
    +          s"failed." +
    +          s"Table [$tableName] either already exists or registered under database [$dbName]")
    +      }
    +    }
    +    finally {
    +      // release lock after command execution completion
    +      CommonUtil.releaseLocks(locks)
    +    }
    +    // update the schema modified time
    +    metaStore.updateAndTouchSchemasUpdatedTime()
    +    Seq.empty
    +  }
    +
    +  /**
    +   * the method prepare the data type for raw column
    +   *
    +   * @param column
    +   * @return
    +   */
    +  def prepareDataType(column: ColumnSchema): String = {
    +    column.getDataType.getName.toLowerCase() match {
    +      case "decimal" =>
    +        "decimal(" + column.getPrecision + "," + column.getScale + ")"
    +      case others =>
    +        others
    +    }
    +  }
    +
    +  /**
    +   * The method register the carbon table with hive
    +   *
    +   * @param dbName
    +   * @param tableName
    +   * @param tableInfo
    +   * @param sparkSession
    +   * @return
    +   */
    +  def registerTableWithHive(dbName: String,
    +      tableName: String,
    +      tableInfo: TableInfo)(sparkSession: SparkSession): Any = {
    +    val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val carbonSchemaString = carbonMetaStore.generateTableSchemaString(tableInfo,
    +      tableInfo.getOrCreateAbsoluteTableIdentifier())
    +    val tablePath = tableInfo.getTablePath
    +    try {
    +      val columns = tableInfo.getFactTable.getListOfColumns.asScala.filter(!_.isInvisible)
    +      // TODO adding the complex type fields should be handled
    +      // getAll Fields
    +      val fields = new Array[String](columns.size)
    +      columns.map(column => fields(column.getSchemaOrdinal)
    +        = column.getColumnName + ' ' + prepareDataType(column))
    +      val operationContext = new OperationContext
    +      val createTablePreExecutionEvent: RefreshTablePreExecutionEvent =
    +        new RefreshTablePreExecutionEvent(sparkSession,
    +          tableInfo.getOrCreateAbsoluteTableIdentifier().getCarbonTableIdentifier,
    +          tablePath)
    +      OperationListenerBus.getInstance.fireEvent(createTablePreExecutionEvent, operationContext)
    +      sparkSession.sql(
    --- End diff --
    
    I think you  can directly CreateTableCOmmand here instead of creating sql here.


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/441/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154900365
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala ---
    @@ -81,12 +90,10 @@ case class CarbonCreateTableCommand(
         val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tableIdentifier)
         if (createDSTable) {
           try {
    -        val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
    -        cm.dimCols.foreach(f => fields(f.schemaOrdinal) = f)
    -        cm.msrCols.foreach(f => fields(f.schemaOrdinal) = f)
    -
    -        sparkSession.sparkContext.setLocalProperty(EXECUTION_ID_KEY, null)
             val tablePath = tableIdentifier.getTablePath
    +        val carbonRelation = CarbonSparkUtil.createCarbonRelation(tableInfo, tablePath)
    +        val fields: Array[Field] = CarbonSparkUtil.getColumnSchemaFields(carbonRelation)
    --- End diff --
    
    you don't require fields here, you just require rawSchema


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/555/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1717/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/452/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/413/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/557/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    retest this please


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/418/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154276544
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala ---
    @@ -193,6 +197,32 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
             ExecutedCommandExec(
               CarbonAlterTableUnsetCommand(tableName, propKeys, ifExists, isView)) :: Nil
           }
    +      case RefreshTable(tableIdentifier) =>
    +        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
    --- End diff --
    
    Fixed


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154566404
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,216 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = GetDB.getDatabaseName(dbName, sparkSession)
    +    val databaseLocation = GetDB.getDatabaseLocation(databaseName, sparkSession,
    +      CarbonProperties.getStorePath)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName
    +    val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    if (!sparkSession.sessionState.catalog.listTables(databaseName)
    +      .exists(_.table.equalsIgnoreCase(tableName))) {
    +      val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
    +      // check the existence of the schema file to know its a carbon table
    +      val schemaFilePath = carbonTablePath.getSchemaFilePath
    +      // if schema file does not exist then the table will either non carbon table or stale
    +      // carbon table
    +      if (FileFactory.isFileExist(schemaFilePath, FileFactory.getFileType(schemaFilePath))) {
    +        // read TableInfo
    +        val tableInfo = metaStore.getWrapperTableInfo(absoluteTableIdentifier)(sparkSession)
    +        // 2.2 register the table with the hive check if the table being registered has
    +        // aggregate table then do the below steps
    +        // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +        val dataMapSchemaList = tableInfo.getDataMapSchemaList
    +        if (null != dataMapSchemaList && dataMapSchemaList.size() != 0) {
    +          // validate all the aggregate tables are copied at the storeLocation
    +          val allExists = validateAllAggregateTablePresent(databaseName,
    +            dataMapSchemaList, databaseLocation)
    +          if (!allExists) {
    +            // fail the register operation
    +            val msg = s"Table registration with Database name [$databaseName] and Table name " +
    +                      s"[$tableName] failed. All the aggregate Tables for table [$tableName] is" +
    --- End diff --
    
    Better print the agg table names which are not copied 


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2008/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155046109
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala ---
    @@ -142,6 +142,10 @@ trait CarbonMetaStore {
     
       def getTableFromMetadataCache(database: String, tableName: String): Option[CarbonTable]
     
    +  def getTableInfo(absoluteTableIdentifier: AbsoluteTableIdentifier)
    --- End diff --
    
    many places we are reading schema from the tablePath, ie we decided to move it at common place.
    Once this PR is merged will refactor code from other places to use this interface.


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155044945
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala ---
    @@ -29,25 +29,34 @@ import org.apache.carbondata.common.logging.LogServiceFactory
     import org.apache.carbondata.core.constants.CarbonCommonConstants
     import org.apache.carbondata.core.exception.InvalidConfigurationException
     import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
    -import org.apache.carbondata.core.metadata.schema.table.TableInfo
    +import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
     import org.apache.carbondata.core.util.CarbonUtil
     import org.apache.carbondata.events.{CreateTablePostExecutionEvent, CreateTablePreExecutionEvent, OperationContext, OperationListenerBus}
    +import org.apache.carbondata.spark.util.CarbonSparkUtil
     
     case class CarbonCreateTableCommand(
    -    cm: TableModel,
    +    tableInfo: TableInfo,
    --- End diff --
    
    correct if user does not pass the database name while creating table, then database name will not 
    there that is why added the following code in CarbonCreateTableCommand.processMetadata
      var databaseOpt : Option[String] = None
        if(tableInfo.getDatabaseName != null) {
          databaseOpt = Some(tableInfo.getDatabaseName)
        }
        val dbName = CarbonEnv.getDatabaseName(databaseOpt)(sparkSession)


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1551/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1550/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/554/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1946/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154118475
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,264 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{CreateTablePreExecutionEvent, OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +import org.apache.carbondata.spark.util.CommonUtil
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = GetDB.getDatabaseName(dbName, sparkSession)
    +    val databaseLocation = GetDB.getDatabaseLocation(databaseName, sparkSession,
    +      CarbonProperties.getStorePath)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
    +    var locks = List.empty[ICarbonLock]
    +    try {
    +      val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName
    +      val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
    +      locks = CommonUtil.acquireLock(locksToBeAcquired, absoluteTableIdentifier)
    +      // 2.1 check if the table already register with hive then ignore and continue with the next
    +      // schema
    +      if (!sparkSession.sessionState.catalog.listTables(databaseName)
    +        .exists(_.table.equalsIgnoreCase(tableName))) {
    +        val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
    --- End diff --
    
    Please move reading of TableInfo using tablePath to the CarbonMetaStore


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2000/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155444131
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala ---
    @@ -142,6 +142,10 @@ trait CarbonMetaStore {
     
       def getTableFromMetadataCache(database: String, tableName: String): Option[CarbonTable]
     
    +  def getTableInfo(absoluteTableIdentifier: AbsoluteTableIdentifier)
    --- End diff --
    
    fixed added in schema redear


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155041379
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala ---
    @@ -320,13 +322,8 @@ class AlterTableColumnSchemaGenerator(
     // TODO: move this to carbon store API
     object TableNewProcessor {
       def apply(
    -      cm: TableModel,
    -      identifier: AbsoluteTableIdentifier): TableInfo = {
    -    new TableNewProcessor(
    -      cm,
    -      identifier.getDatabaseName,
    -      identifier.getTableName,
    -      identifier.getTablePath).process
    +      cm: TableModel): TableInfo = {
    --- End diff --
    
    We have to pass the TableInfo to the CarbonCreateTableCommand so that we can use the 
    CarbonCreateTableCommand  flow from the RefreshCarbonTableCommand.
    Any ways for supporting the external table path, the tablePath is being passed to CarbonCreateTableCommand


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/451/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155268568
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala ---
    @@ -29,25 +29,34 @@ import org.apache.carbondata.common.logging.LogServiceFactory
     import org.apache.carbondata.core.constants.CarbonCommonConstants
     import org.apache.carbondata.core.exception.InvalidConfigurationException
     import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
    -import org.apache.carbondata.core.metadata.schema.table.TableInfo
    +import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
     import org.apache.carbondata.core.util.CarbonUtil
     import org.apache.carbondata.events.{CreateTablePostExecutionEvent, CreateTablePreExecutionEvent, OperationContext, OperationListenerBus}
    +import org.apache.carbondata.spark.util.CarbonSparkUtil
     
     case class CarbonCreateTableCommand(
    -    cm: TableModel,
    +    tableInfo: TableInfo,
    --- End diff --
    
    This approach is fine. But why you need to modify it in first place? TableModel can't work?


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154984011
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala ---
    @@ -47,6 +47,7 @@ import org.apache.carbondata.spark.util.DataTypeConverterUtil
     
     case class TableModel(
         ifNotExistsSet: Boolean,
    +    var databaseName: String,
    --- End diff --
    
    There is already a databaseNameOp in line 51


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155444169
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    --- End diff --
    
    fixed change to databaseNameOp


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155038052
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -1044,48 +1045,48 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         dataType match {
           case "string" =>
             Field(field.column, Some("String"), field.name, Some(null), field.parent,
    -        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema
    -      )
    +        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema,
    +        field.columnComment)
           case "smallint" =>
             Field(field.column, Some("SmallInt"), field.name, Some(null),
               field.parent, field.storeType, field.schemaOrdinal,
    -          field.precision, field.scale, field.rawSchema)
    +          field.precision, field.scale, field.rawSchema, field.columnComment)
           case "integer" | "int" =>
             Field(field.column, Some("Integer"), field.name, Some(null),
               field.parent, field.storeType, field.schemaOrdinal,
    -          field.precision, field.scale, field.rawSchema)
    +          field.precision, field.scale, field.rawSchema, field.columnComment)
           case "long" => Field(field.column, Some("Long"), field.name, Some(null), field.parent,
    -        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema
    -      )
    +        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema,
    +        field.columnComment)
           case "double" => Field(field.column, Some("Double"), field.name, Some(null), field.parent,
    -        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema
    -      )
    +        field.storeType, field.schemaOrdinal, field.precision, field.scale, field.rawSchema,
    +        field.columnComment)
    --- End diff --
    
    this is needed as part of this pr only


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r155267373
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala ---
    @@ -142,6 +142,10 @@ trait CarbonMetaStore {
     
       def getTableFromMetadataCache(database: String, tableName: String): Option[CarbonTable]
     
    +  def getTableInfo(absoluteTableIdentifier: AbsoluteTableIdentifier)
    --- End diff --
    
    This idea is good, but I think you do not need to put this in CarbonMetaStore trait, right? Reading schema from tablePath can be an utility.


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/481/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1726/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2096/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/465/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1732/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    It seems there is a lot of change related to column comment, please separate them to another PR if not  related to this PR


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1647/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1742/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    retest sdv please


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    retest this please


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154900160
  
    --- Diff: integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala ---
    @@ -51,4 +55,90 @@ object CarbonSparkUtil {
           table)
       }
     
    +  /**
    +   * Below method will be used to get the fields object for pre aggregate table
    +   *
    +   * @param columnName
    +   * @param dataType
    +   * @param aggregateType
    +   * @param parentColumnId
    +   * @param parentTableName
    +   * @param parentDatabaseName
    +   * @param parentTableId
    +   * @return fields object
    +   */
    +  def getField(columnName: String,
    --- End diff --
    
    Why do you require this code? you ultimately require `rawSchema` , that you can anyway get from `dataType.catalogString`, just append the comment to it and use in createtablecommand


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1534/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1595/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/457/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2183/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    LGTM


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154120941
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala ---
    @@ -193,6 +197,32 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
             ExecutedCommandExec(
               CarbonAlterTableUnsetCommand(tableName, propKeys, ifExists, isView)) :: Nil
           }
    +      case RefreshTable(tableIdentifier) =>
    +        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
    --- End diff --
    
    here just call `RefreshCarbonTableCommand ` and at the end of that command you can call `RefreshTable`


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1721/



---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154276519
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,264 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{CreateTablePreExecutionEvent, OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +import org.apache.carbondata.spark.util.CommonUtil
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = GetDB.getDatabaseName(dbName, sparkSession)
    +    val databaseLocation = GetDB.getDatabaseLocation(databaseName, sparkSession,
    +      CarbonProperties.getStorePath)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
    +    var locks = List.empty[ICarbonLock]
    +    try {
    +      val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName
    +      val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
    +      locks = CommonUtil.acquireLock(locksToBeAcquired, absoluteTableIdentifier)
    +      // 2.1 check if the table already register with hive then ignore and continue with the next
    +      // schema
    +      if (!sparkSession.sessionState.catalog.listTables(databaseName)
    +        .exists(_.table.equalsIgnoreCase(tableName))) {
    +        val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
    +        // check the existence of the schema file to know its a carbon table
    +        val schemaFilePath = carbonTablePath.getSchemaFilePath
    +        // if schema file does not exist then the table will either non carbon table or stale
    +        // carbon table
    +        if (FileFactory.isFileExist(schemaFilePath, FileFactory.getFileType(schemaFilePath))) {
    +          // read TableInfo
    +          val thriftTableInfo: org.apache.carbondata.format.TableInfo = metaStore
    +            .getThriftTableInfo(carbonTablePath)(sparkSession)
    +          val schemaConverter = new ThriftWrapperSchemaConverterImpl()
    +          val tableInfo = schemaConverter
    +            .fromExternalToWrapperTableInfo(thriftTableInfo,
    +              databaseName,
    +              tableName,
    +              absoluteTableIdentifier.getTablePath)
    +          // 2.2 register the table with the hive check if the table being registered has
    +          // aggregate table then do the below steps
    +          // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +          val dataMapSchemaList = tableInfo.getDataMapSchemaList
    +          if (null != dataMapSchemaList && dataMapSchemaList.size() != 0) {
    +            // validate all the aggregate tables are copied at the storeLocation
    +            val allExists = validateAllAggregateTablePresent(databaseName,
    +              dataMapSchemaList, databaseLocation)
    +            if (!allExists) {
    +              // fail the register operation
    +              val msg = s"Table registration with Database name [$databaseName] and Table name " +
    +                        s"[$tableName] failed. All the aggregate Tables for table [$tableName] is" +
    +                        s" not copied under database [$databaseName]"
    +              LOGGER.audit(msg)
    +              CarbonException.analysisException(msg)
    +            }
    +            // 2.2.1 Register the aggregate tables to hive
    +            registerAggregates(databaseName, dataMapSchemaList, databaseLocation)(sparkSession)
    +          }
    +          registerTableWithHive(databaseName, tableName, tableInfo)(sparkSession)
    +        } else {
    +          LOGGER.audit(
    +            s"Table registration with Database name [$databaseName] and Table name [$tableName] " +
    +            s"failed." +
    +            s"Table [$tableName] either non carbon table or stale carbon table under database " +
    +            s"[$databaseName]")
    +        }
    +      } else {
    +        LOGGER.audit(
    +          s"Table registration with Database name [$databaseName] and Table name [$tableName] " +
    +          s"failed." +
    +          s"Table [$tableName] either already exists or registered under database [$dbName]")
    +      }
    +    }
    +    finally {
    +      // release lock after command execution completion
    +      CommonUtil.releaseLocks(locks)
    +    }
    +    // update the schema modified time
    +    metaStore.updateAndTouchSchemasUpdatedTime()
    +    Seq.empty
    +  }
    +
    +  /**
    +   * the method prepare the data type for raw column
    +   *
    +   * @param column
    +   * @return
    +   */
    +  def prepareDataType(column: ColumnSchema): String = {
    +    column.getDataType.getName.toLowerCase() match {
    +      case "decimal" =>
    +        "decimal(" + column.getPrecision + "," + column.getScale + ")"
    +      case others =>
    +        others
    +    }
    +  }
    +
    +  /**
    +   * The method register the carbon table with hive
    +   *
    +   * @param dbName
    +   * @param tableName
    +   * @param tableInfo
    +   * @param sparkSession
    +   * @return
    +   */
    +  def registerTableWithHive(dbName: String,
    +      tableName: String,
    +      tableInfo: TableInfo)(sparkSession: SparkSession): Any = {
    +    val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val carbonSchemaString = carbonMetaStore.generateTableSchemaString(tableInfo,
    +      tableInfo.getOrCreateAbsoluteTableIdentifier())
    +    val tablePath = tableInfo.getTablePath
    +    try {
    +      val columns = tableInfo.getFactTable.getListOfColumns.asScala.filter(!_.isInvisible)
    +      // TODO adding the complex type fields should be handled
    +      // getAll Fields
    +      val fields = new Array[String](columns.size)
    +      columns.map(column => fields(column.getSchemaOrdinal)
    +        = column.getColumnName + ' ' + prepareDataType(column))
    +      val operationContext = new OperationContext
    +      val createTablePreExecutionEvent: RefreshTablePreExecutionEvent =
    +        new RefreshTablePreExecutionEvent(sparkSession,
    +          tableInfo.getOrCreateAbsoluteTableIdentifier().getCarbonTableIdentifier,
    +          tablePath)
    +      OperationListenerBus.getInstance.fireEvent(createTablePreExecutionEvent, operationContext)
    +      sparkSession.sql(
    --- End diff --
    
    @ravipesala Please elaborate how to use the same i don't see any instance.


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by mohammadshahidkhan <gi...@git.apache.org>.
Github user mohammadshahidkhan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154276652
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,264 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{CreateTablePreExecutionEvent, OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +import org.apache.carbondata.spark.util.CommonUtil
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = GetDB.getDatabaseName(dbName, sparkSession)
    +    val databaseLocation = GetDB.getDatabaseLocation(databaseName, sparkSession,
    +      CarbonProperties.getStorePath)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
    --- End diff --
    
    removed 


---

[GitHub] carbondata pull request #1583: [CARBONDATA-1822][Spark-Integration] Support ...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1583#discussion_r154984765
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala ---
    @@ -0,0 +1,213 @@
    +/*
    + * 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.spark.sql.execution.command.management
    +
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.execution.command.MetadataCommand
    +import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
    +import org.apache.spark.sql.util.CarbonException
    +
    +import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.datastore.impl.FileFactory
    +import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
    +import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
    +import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.core.util.path.CarbonStorePath
    +import org.apache.carbondata.events.{OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
    +
    +/**
    + * Command to register carbon table from existing carbon table data
    + */
    +case class RefreshCarbonTableCommand(
    +    dbName: Option[String],
    +    tableName: String)
    +  extends MetadataCommand {
    +  val LOGGER: LogService =
    +    LogServiceFactory.getLogService(this.getClass.getName)
    +
    +  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
    +    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
    +    val databaseName = CarbonEnv.getDatabaseName(dbName)(sparkSession)
    +    val databaseLocation = CarbonEnv.getDatabaseLocation(databaseName, sparkSession)
    +    // Steps
    +    // 1. get table path
    +    // 2. perform the below steps
    +    // 2.1 check if the table already register with hive then ignore and continue with the next
    +    // schema
    +    // 2.2 register the table with the hive check if the table being registered has aggregate table
    +    // then do the below steps
    +    // 2.2.1 validate that all the aggregate tables are copied at the store location.
    +    // 2.2.2 Register the aggregate tables
    +    val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName
    --- End diff --
    
    use `CarbonEnv.getTablePath` to get the path


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2076/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2036/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/480/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2002/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1545/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    retest this please


---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1938/



---

[GitHub] carbondata issue #1583: [CARBONDATA-1822][Spark-Integration] Support DDL to ...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1583
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/2091/



---