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

[GitHub] carbondata pull request #1488: [CARBONDATA-1528] Alter preaggreagate support

GitHub user kunal642 opened a pull request:

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

    [CARBONDATA-1528] Alter preaggreagate support

    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [X] Any interfaces changed?
     
     - [X] Any backward compatibility impacted?
     
     - [X] Document update required?
    
     - [X] 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.
           
     - [X] 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/kunal642/carbondata alter_preaggreagate_support

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

    https://github.com/apache/carbondata/pull/1488.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 #1488
    
----
commit 47087506b1d219a44100ac04dcc855f3dd2aad07
Author: kunal642 <ku...@gmail.com>
Date:   2017-10-23T12:58:15Z

    restrict update/delete for pre-aggregate table

commit e9960d0c6b4065d8e550d762f43591c114907a4d
Author: kunal642 <ku...@gmail.com>
Date:   2017-10-23T12:58:15Z

    added support for alter operations

----


---

[GitHub] carbondata pull request #1488: [CARBONDATA-1528] [PreAgg] Add validation for...

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

    https://github.com/apache/carbondata/pull/1488#discussion_r150380968
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/listeners/PreAggregateDataTypeChangePreListener.scala ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.preaaggregate.listeners
    --- End diff --
    
    move all the validation listners related preagg move to one scala file. It will be more readable with out confusion


---

[GitHub] carbondata issue #1488: [CARBONDATA-1528] [PreAgg] Add validation for alter ...

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

    https://github.com/apache/carbondata/pull/1488
  
    @ravipesala Please review


---

[GitHub] carbondata issue #1488: [CARBONDATA-1528] [PreAgg] Add validation for alter ...

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

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



---

[GitHub] carbondata issue #1488: [CARBONDATA-1528] [PreAgg] Add validation for alter ...

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

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



---

[GitHub] carbondata issue #1488: [CARBONDATA-1528] [PreAgg] Add validation for alter ...

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

    https://github.com/apache/carbondata/pull/1488
  
    Please refer to #1448 . Add a strategy for pre-agg table and add checks there


---

[GitHub] carbondata issue #1488: [CARBONDATA-1528] [PreAgg] Add validation for alter ...

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

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



---

[GitHub] carbondata issue #1488: [CARBONDATA-1528] Alter preaggreagate support

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

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



---

[GitHub] carbondata pull request #1488: [CARBONDATA-1528] [PreAgg] Add validation for...

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

    https://github.com/apache/carbondata/pull/1488#discussion_r150380951
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/listeners/PreAggregateDataTypeChangePreListener.scala ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.preaaggregate.listeners
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.carbondata.events.{AlterTableDataTypeChangePreEvent, Event, OperationContext, OperationEventListener}
    +
    +object PreAggregateDataTypeChangePreListener extends OperationEventListener {
    +  /**
    +   * Called on a specified event occurrence
    +   *
    +   * @param event
    +   * @param operationContext
    +   */
    +  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
    +    val dataTypeChangePreListener = event.asInstanceOf[AlterTableDataTypeChangePreEvent]
    +    val carbonTable = dataTypeChangePreListener.carbonTable
    +    val alterTableDataTypeChangeModel = dataTypeChangePreListener.alterTableDataTypeChangeModel
    +    val columnToBeAltered: String = alterTableDataTypeChangeModel.columnName
    +    val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList
    +    if (dataMapSchemas != null && !dataMapSchemas.isEmpty) {
    +      dataMapSchemas.asScala.foreach {
    +        dataMapSchema =>
    --- End diff --
    
    move to above line


---

[GitHub] carbondata pull request #1488: [CARBONDATA-1528] [PreAgg] Add validation for...

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

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


---

[GitHub] carbondata issue #1488: [CARBONDATA-1528] [PreAgg] Add validation for alter ...

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

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



---

[GitHub] carbondata pull request #1488: [CARBONDATA-1528] [PreAgg] Add validation for...

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

    https://github.com/apache/carbondata/pull/1488#discussion_r150380999
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/listeners/PreAggregateRenameTablePostListener.scala ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.preaaggregate.listeners
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.{CarbonEnv, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.execution.command.AlterTableRenameModel
    +import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
    +
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
    +import org.apache.carbondata.events.{AlterTableRenamePostEvent, Event, OperationContext, OperationEventListener}
    +import org.apache.carbondata.format.SchemaEvolutionEntry
    +
    +object PreAggregateRenameTablePostListener extends OperationEventListener {
    +
    +  /**
    +   * Called on a specified event occurrence
    +   *
    +   * @param event
    +   * @param operationContext
    +   */
    +  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
    +    val renameTablePostListener = event.asInstanceOf[AlterTableRenamePostEvent]
    +    val carbonTable = renameTablePostListener.carbonTable
    +    implicit val sparkSession: SparkSession = renameTablePostListener.sparkSession
    +    val renameTableModel = renameTablePostListener.alterTableRenameModel
    +    val oldParentTableName = renameTableModel.oldTableIdentifier.table
    +    val oldParentDatabaseName = renameTableModel.oldTableIdentifier.database.getOrElse("default")
    +    val newParentTableName = renameTableModel.newTableIdentifier.table
    +    if (carbonTable.hasPreAggregateTables) {
    +      val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList.asScala
    +      dataMapSchemas.foreach {
    +        dataMapSchema =>
    +          val childTableIdentifier = dataMapSchema.getRelationIdentifier
    +          val childCarbonTable = PreAggregateUtil
    +            .getChildCarbonTable(childTableIdentifier.getDatabaseName,
    +              childTableIdentifier.getTableName)(sparkSession)
    +          updateChildTableWithNewParent(renameTableModel, childCarbonTable)(sparkSession)
    +      }
    +    }
    +    val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +    schemaEvolutionEntry.setTime_stamp(System.currentTimeMillis())
    +    updateParentRelationIdentifierForColumns(carbonTable.getTableInfo,
    +      oldParentDatabaseName,
    +      oldParentTableName,
    +      newParentTableName)
    +    updateTableSchema(carbonTable,
    +      schemaEvolutionEntry,
    +      oldParentDatabaseName,
    +      oldParentTableName)
    +  }
    +
    +  private def updateChildTableWithNewParent(renameTableModel: AlterTableRenameModel,
    +      childCarbonTable: Option[CarbonTable])(implicit sparkSession: SparkSession): Unit = {
    +    val oldParentTableName = renameTableModel.oldTableIdentifier.table
    +    val oldParentDatabaseName = renameTableModel.oldTableIdentifier.database.getOrElse("default")
    +    val newParentTableName = renameTableModel.newTableIdentifier.table
    +    if (childCarbonTable.isDefined) {
    +      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +      schemaEvolutionEntry.setTime_stamp(System.currentTimeMillis())
    +      val childTableInfo = childCarbonTable.get.getTableInfo
    +      childTableInfo.getParentRelationIdentifiers.asScala.foreach {
    +        parentRelationIdentifier =>
    +          if (parentRelationIdentifier.getDatabaseName
    +                .equalsIgnoreCase(oldParentDatabaseName) &&
    +              parentRelationIdentifier.getTableName
    +                .equalsIgnoreCase(oldParentTableName)) {
    +            parentRelationIdentifier.setTableName(newParentTableName)
    +          }
    +      }
    +      updateTableSchema(childCarbonTable.get,
    +        schemaEvolutionEntry,
    +        childTableInfo.getDatabaseName,
    +        childTableInfo.getFactTable.getTableName)
    +    }
    +  }
    +
    +  private def updateParentRelationIdentifierForColumns(parentTableInfo: TableInfo,
    +      oldParentDatabaseName: String,
    +      oldParentTableName: String,
    +      newParentTableName: String): Unit = {
    +    parentTableInfo.getFactTable.getListOfColumns.asScala.foreach {
    +      columnSchema =>
    --- End diff --
    
    move to above line


---

[GitHub] carbondata pull request #1488: [CARBONDATA-1528] [PreAgg] Add validation for...

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

    https://github.com/apache/carbondata/pull/1488#discussion_r150406795
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/listeners/PreAggregateRenameTablePostListener.scala ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.preaaggregate.listeners
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.{CarbonEnv, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.execution.command.AlterTableRenameModel
    +import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
    +
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
    +import org.apache.carbondata.events.{AlterTableRenamePostEvent, Event, OperationContext, OperationEventListener}
    +import org.apache.carbondata.format.SchemaEvolutionEntry
    +
    +object PreAggregateRenameTablePostListener extends OperationEventListener {
    +
    +  /**
    +   * Called on a specified event occurrence
    +   *
    +   * @param event
    +   * @param operationContext
    +   */
    +  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
    +    val renameTablePostListener = event.asInstanceOf[AlterTableRenamePostEvent]
    +    val carbonTable = renameTablePostListener.carbonTable
    +    implicit val sparkSession: SparkSession = renameTablePostListener.sparkSession
    +    val renameTableModel = renameTablePostListener.alterTableRenameModel
    +    val oldParentTableName = renameTableModel.oldTableIdentifier.table
    +    val oldParentDatabaseName = renameTableModel.oldTableIdentifier.database.getOrElse("default")
    +    val newParentTableName = renameTableModel.newTableIdentifier.table
    +    if (carbonTable.hasPreAggregateTables) {
    +      val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList.asScala
    +      dataMapSchemas.foreach {
    +        dataMapSchema =>
    +          val childTableIdentifier = dataMapSchema.getRelationIdentifier
    +          val childCarbonTable = PreAggregateUtil
    +            .getChildCarbonTable(childTableIdentifier.getDatabaseName,
    +              childTableIdentifier.getTableName)(sparkSession)
    +          updateChildTableWithNewParent(renameTableModel, childCarbonTable)(sparkSession)
    +      }
    +    }
    +    val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +    schemaEvolutionEntry.setTime_stamp(System.currentTimeMillis())
    +    updateParentRelationIdentifierForColumns(carbonTable.getTableInfo,
    +      oldParentDatabaseName,
    +      oldParentTableName,
    +      newParentTableName)
    +    updateTableSchema(carbonTable,
    +      schemaEvolutionEntry,
    +      oldParentDatabaseName,
    +      oldParentTableName)
    +  }
    +
    +  private def updateChildTableWithNewParent(renameTableModel: AlterTableRenameModel,
    +      childCarbonTable: Option[CarbonTable])(implicit sparkSession: SparkSession): Unit = {
    +    val oldParentTableName = renameTableModel.oldTableIdentifier.table
    +    val oldParentDatabaseName = renameTableModel.oldTableIdentifier.database.getOrElse("default")
    +    val newParentTableName = renameTableModel.newTableIdentifier.table
    +    if (childCarbonTable.isDefined) {
    +      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +      schemaEvolutionEntry.setTime_stamp(System.currentTimeMillis())
    +      val childTableInfo = childCarbonTable.get.getTableInfo
    +      childTableInfo.getParentRelationIdentifiers.asScala.foreach {
    +        parentRelationIdentifier =>
    +          if (parentRelationIdentifier.getDatabaseName
    +                .equalsIgnoreCase(oldParentDatabaseName) &&
    +              parentRelationIdentifier.getTableName
    +                .equalsIgnoreCase(oldParentTableName)) {
    +            parentRelationIdentifier.setTableName(newParentTableName)
    +          }
    +      }
    +      updateTableSchema(childCarbonTable.get,
    +        schemaEvolutionEntry,
    +        childTableInfo.getDatabaseName,
    +        childTableInfo.getFactTable.getTableName)
    +    }
    +  }
    +
    +  private def updateParentRelationIdentifierForColumns(parentTableInfo: TableInfo,
    +      oldParentDatabaseName: String,
    +      oldParentTableName: String,
    +      newParentTableName: String): Unit = {
    +    parentTableInfo.getFactTable.getListOfColumns.asScala.foreach {
    +      columnSchema =>
    --- End diff --
    
    Ok, please block it for now.


---

[GitHub] carbondata pull request #1488: [CARBONDATA-1528] [PreAgg] Add validation for...

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

    https://github.com/apache/carbondata/pull/1488#discussion_r150381324
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/listeners/PreAggregateRenameTablePostListener.scala ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.preaaggregate.listeners
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.{CarbonEnv, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.execution.command.AlterTableRenameModel
    +import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
    +
    +import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
    +import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
    +import org.apache.carbondata.events.{AlterTableRenamePostEvent, Event, OperationContext, OperationEventListener}
    +import org.apache.carbondata.format.SchemaEvolutionEntry
    +
    +object PreAggregateRenameTablePostListener extends OperationEventListener {
    +
    +  /**
    +   * Called on a specified event occurrence
    +   *
    +   * @param event
    +   * @param operationContext
    +   */
    +  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
    +    val renameTablePostListener = event.asInstanceOf[AlterTableRenamePostEvent]
    +    val carbonTable = renameTablePostListener.carbonTable
    +    implicit val sparkSession: SparkSession = renameTablePostListener.sparkSession
    +    val renameTableModel = renameTablePostListener.alterTableRenameModel
    +    val oldParentTableName = renameTableModel.oldTableIdentifier.table
    +    val oldParentDatabaseName = renameTableModel.oldTableIdentifier.database.getOrElse("default")
    +    val newParentTableName = renameTableModel.newTableIdentifier.table
    +    if (carbonTable.hasPreAggregateTables) {
    +      val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList.asScala
    +      dataMapSchemas.foreach {
    +        dataMapSchema =>
    +          val childTableIdentifier = dataMapSchema.getRelationIdentifier
    +          val childCarbonTable = PreAggregateUtil
    +            .getChildCarbonTable(childTableIdentifier.getDatabaseName,
    +              childTableIdentifier.getTableName)(sparkSession)
    +          updateChildTableWithNewParent(renameTableModel, childCarbonTable)(sparkSession)
    +      }
    +    }
    +    val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +    schemaEvolutionEntry.setTime_stamp(System.currentTimeMillis())
    +    updateParentRelationIdentifierForColumns(carbonTable.getTableInfo,
    +      oldParentDatabaseName,
    +      oldParentTableName,
    +      newParentTableName)
    +    updateTableSchema(carbonTable,
    +      schemaEvolutionEntry,
    +      oldParentDatabaseName,
    +      oldParentTableName)
    +  }
    +
    +  private def updateChildTableWithNewParent(renameTableModel: AlterTableRenameModel,
    +      childCarbonTable: Option[CarbonTable])(implicit sparkSession: SparkSession): Unit = {
    +    val oldParentTableName = renameTableModel.oldTableIdentifier.table
    +    val oldParentDatabaseName = renameTableModel.oldTableIdentifier.database.getOrElse("default")
    +    val newParentTableName = renameTableModel.newTableIdentifier.table
    +    if (childCarbonTable.isDefined) {
    +      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +      schemaEvolutionEntry.setTime_stamp(System.currentTimeMillis())
    +      val childTableInfo = childCarbonTable.get.getTableInfo
    +      childTableInfo.getParentRelationIdentifiers.asScala.foreach {
    +        parentRelationIdentifier =>
    +          if (parentRelationIdentifier.getDatabaseName
    +                .equalsIgnoreCase(oldParentDatabaseName) &&
    +              parentRelationIdentifier.getTableName
    +                .equalsIgnoreCase(oldParentTableName)) {
    +            parentRelationIdentifier.setTableName(newParentTableName)
    +          }
    +      }
    +      updateTableSchema(childCarbonTable.get,
    +        schemaEvolutionEntry,
    +        childTableInfo.getDatabaseName,
    +        childTableInfo.getFactTable.getTableName)
    +    }
    +  }
    +
    +  private def updateParentRelationIdentifierForColumns(parentTableInfo: TableInfo,
    +      oldParentDatabaseName: String,
    +      oldParentTableName: String,
    +      newParentTableName: String): Unit = {
    +    parentTableInfo.getFactTable.getListOfColumns.asScala.foreach {
    +      columnSchema =>
    --- End diff --
    
    @ravipesala As discussed with vishal, rename table has a lot ol places that need to be updated with new parent name.
    
    1. All the child tables have to be updated.
    2. All parentColumnIdentifiers have to be updated with the new table
    3. table properties map has to be updated for all childs.
    4. the select query has to be updated with new parent for all childs.
    
    Is it possible to block rename too?


---

[GitHub] carbondata issue #1488: [CARBONDATA-1528] [PreAgg] Add validation for alter ...

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

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



---