[GitHub] incubator-carbondata pull request #641: [WIP] Alter table support for carbon...

classic Classic list List threaded Threaded
73 messages Options
1234
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106090749
 
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala ---
    @@ -784,4 +786,105 @@ object GlobalDictionaryUtil {
             throw ex
         }
       }
    +
    +  def loadDefaultDictionaryValueForNewColumn(carbonTablePath: CarbonTablePath,
    --- End diff --
   
    Modify the previous flow to use common function


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106090051
 
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala ---
    @@ -194,4 +196,102 @@ object CarbonScalaUtil {
           }
         }
       }
    +
    +  /**
    +   * This method will validate a column for its data type and check whether the column data type
    +   * can be modified and update if conditions are met
    +   *
    +   * @param dataTypeInfo
    +   * @param carbonColumn
    +   */
    +  def validateColumnDataType(dataTypeInfo: DataTypeInfo, carbonColumn: CarbonColumn): Unit = {
    +    carbonColumn.getDataType.getName match {
    +      case "INT" =>
    +        if (!dataTypeInfo.dataType.equals("bigint")) {
    +          sys
    +            .error(s"Given column ${ carbonColumn.getColName } with data type ${
    +              carbonColumn
    +                .getDataType.getName
    +            } cannot be modified. Int can only be changed to bigInt")
    +        }
    +      case "DECIMAL" =>
    +        if (!dataTypeInfo.dataType.equals("decimal")) {
    +          sys
    +            .error(s"Given column ${ carbonColumn.getColName } with data type ${
    +              carbonColumn.getDataType.getName
    +            } cannot be modified. Decimal can be only be changed to Decimal of higher precision")
    +        }
    +        if (dataTypeInfo.precision <= carbonColumn.getColumnSchema.getPrecision) {
    +          sys
    +            .error(s"Given column ${
    +              carbonColumn
    +                .getColName
    +            } cannot be modified. Specified precision value ${
    +              dataTypeInfo
    +                .precision
    +            } should be greater or equal to current precision value ${
    +              carbonColumn.getColumnSchema
    +                .getPrecision
    +            }")
    +        } else if (dataTypeInfo.scale <= carbonColumn.getColumnSchema.getScale) {
    +          sys
    +            .error(s"Given column ${
    +              carbonColumn
    +                .getColName
    +            } cannot be modified. Specified scale value ${
    +              dataTypeInfo
    +                .scale
    +            } should be greater or equal to current scale value ${
    +              carbonColumn.getColumnSchema
    +                .getScale
    +            }")
    +        } else {
    +          // difference of precision and scale specified by user should not be less than the
    +          // difference of already existing precision and scale else it will result in data loss
    +          val carbonColumnPrecisionScaleDiff = carbonColumn.getColumnSchema.getPrecision -
    +                                               carbonColumn.getColumnSchema.getScale
    +          val dataInfoPrecisionScaleDiff = dataTypeInfo.precision - dataTypeInfo.scale
    +          if (dataInfoPrecisionScaleDiff < carbonColumnPrecisionScaleDiff) {
    +            sys
    +              .error(s"Given column ${
    +                carbonColumn
    +                  .getColName
    +              } cannot be modified. Specified precision and scale values will lead to data loss")
    +          }
    +        }
    +      case _ =>
    +        sys
    +          .error(s"Given column ${ carbonColumn.getColName } with data type ${
    +            carbonColumn
    +              .getDataType.getName
    +          } cannot be modified. Only Int and Decimal data types are allowed for modification")
    +    }
    +  }
    +
    +  /**
    +   * This method will create a copy of the same object
    +   *
    +   * @param thriftColumnSchema object to be cloned
    +   * @return
    +   */
    +  def createColumnSchemaCopyObject(thriftColumnSchema: org.apache.carbondata.format.ColumnSchema)
    --- End diff --
   
    Use schema copy will work, instead of writing new copy method


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106095089
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -136,6 +140,298 @@ case class AlterTableCompaction(alterTableModel: AlterTableModel) extends Runnab
       }
     }
     
    +private[sql] case class AlterTableDataTypeChange(
    +    alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel) extends RunnableCommand {
    +
    +  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
    +
    +  def run(sparkSession: SparkSession): Seq[Row] = {
    +    val tableName = alterTableDataTypeChangeModel.tableName
    +    val dbName = alterTableDataTypeChangeModel.databaseName
    +      .getOrElse(sparkSession.catalog.currentDatabase)
    +    LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
    +    val relation =
    +      CarbonEnv.get.carbonMetastore
    +        .lookupRelation(Option(dbName), tableName)(sparkSession)
    +        .asInstanceOf[CarbonRelation]
    +    if (relation == null) {
    +      LOGGER.audit(s"Alter table change data type request has failed. " +
    +                   s"Table $dbName.$tableName does not exist")
    +      sys.error(s"Table $dbName.$tableName does not exist")
    +    }
    +    // acquire the lock first
    +    val table = relation.tableMeta.carbonTable
    +    val carbonLock = CarbonLockFactory
    +      .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
    +        LockUsage.METADATA_LOCK)
    +    try {
    +      // get the latest carbon table and check for column existence
    +      val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
    +      val columnName = alterTableDataTypeChangeModel.columnName
    +      var carbonColumnToBeModified: CarbonColumn = null
    +      val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala
    +      // read the latest schema file
    +      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
    +        carbonTable.getCarbonTableIdentifier)
    +      val tableMetadataFile = carbonTablePath.getSchemaFilePath
    +      val tableInfo: org.apache.carbondata.format.TableInfo = CarbonEnv.get.carbonMetastore
    +        .readSchemaFile(tableMetadataFile)
    +      // maintain the added column for schema evolution history
    +      var addColumnSchema: org.apache.carbondata.format.ColumnSchema = null
    +      var deletedColumnSchema: org.apache.carbondata.format.ColumnSchema = null
    +      val columnSchemaList = tableInfo.fact_table.table_columns.asScala
    +      columnSchemaList.foreach { columnSchema =>
    +        if (columnSchema.column_name.equalsIgnoreCase(columnName)) {
    +          deletedColumnSchema = CarbonScalaUtil.createColumnSchemaCopyObject(columnSchema)
    +          columnSchema.setData_type(DataTypeConverterUtil
    +            .convertToThriftDataType(alterTableDataTypeChangeModel.dataTypeInfo.dataType))
    +          columnSchema.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
    +          columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
    +          addColumnSchema = columnSchema
    +        }
    +      }
    +      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +      schemaEvolutionEntry.setAdded(List(addColumnSchema).asJava)
    +      schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava)
    +      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
    +        .setTime_stamp(System.currentTimeMillis)
    +      CarbonEnv.get.carbonMetastore
    +        .updateTableSchema(carbonTable.getCarbonTableIdentifier,
    +          tableInfo,
    +          schemaEvolutionEntry,
    +          carbonTable.getStorePath)(sparkSession)
    +
    +      val tableIdentifier = TableIdentifier(tableName, Some(dbName))
    +      val schema = CarbonEnv.get.carbonMetastore
    +        .lookupRelation(tableIdentifier)(sparkSession).schema.json
    +      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.runSqlHive(
    +        s"ALTER TABLE $dbName.$tableName SET TBLPROPERTIES('spark.sql.sources.schema'='$schema')")
    +      sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
    +      LOGGER.info(s"Alter table for data type change is successful for table $dbName.$tableName")
    +      LOGGER.audit(s"Alter table for data type change is successful for table $dbName.$tableName")
    +    } catch {
    +      case e: Exception =>
    +        LOGGER.error("Alter table change datatype failed : " + e.getMessage)
    +        throw e
    +    } finally {
    +      // release lock after command execution completion
    +      if (carbonLock != null) {
    +        if (carbonLock.unlock()) {
    +          LOGGER.info("Alter table change data type lock released successfully")
    +        } else {
    +          LOGGER.error("Unable to release lock during alter table change data type operation")
    +        }
    +      }
    +    }
    +    Seq.empty
    +  }
    +}
    +
    +private[sql] case class AlterTableAddColumns(
    +    alterTableAddColumnsModel: AlterTableAddColumnsModel) extends RunnableCommand {
    +
    +  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
    +
    +  def run(sparkSession: SparkSession): Seq[Row] = {
    +    val tableName = alterTableAddColumnsModel.tableName
    +    val dbName = alterTableAddColumnsModel.databaseName
    +      .getOrElse(sparkSession.catalog.currentDatabase)
    +    LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
    +    val relation =
    +      CarbonEnv.get.carbonMetastore
    +        .lookupRelation(Option(dbName), tableName)(sparkSession)
    +        .asInstanceOf[CarbonRelation]
    +    if (relation == null) {
    +      LOGGER.audit(s"Alter table add columns request has failed. " +
    +                   s"Table $dbName.$tableName does not exist")
    +      sys.error(s"Table $dbName.$tableName does not exist")
    +    }
    +    // acquire the lock first
    +    val table = relation.tableMeta.carbonTable
    +    val carbonLock = CarbonLockFactory
    +      .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
    +        LockUsage.METADATA_LOCK)
    +    try {
    +      // get the latest carbon table and check for column existence
    +      val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
    +      // read the latest schema file
    +      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
    +        carbonTable.getCarbonTableIdentifier)
    +      val tableMetadataFile = carbonTablePath.getSchemaFilePath
    +      val thriftTableInfo: org.apache.carbondata.format.TableInfo = CarbonEnv.get.carbonMetastore
    +        .readSchemaFile(tableMetadataFile)
    +      val schemaConverter = new ThriftWrapperSchemaConverterImpl()
    +      val wrapperTableInfo = schemaConverter
    +        .fromExternalToWrapperTableInfo(thriftTableInfo,
    +          dbName,
    +          tableName,
    +          carbonTable.getStorePath)
    +      val newCols = new AlterTableProcessor(alterTableAddColumnsModel,
    +        dbName,
    +        wrapperTableInfo,
    +        carbonTablePath,
    +        carbonTable.getCarbonTableIdentifier,
    +        carbonTable.getStorePath).process
    +      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata
    +      .schema.SchemaEvolutionEntry()
    +      schemaEvolutionEntry.setTimeStamp(System.currentTimeMillis)
    +      schemaEvolutionEntry.setAdded(newCols.toList.asJava)
    +
    +      val thriftTable = schemaConverter
    +        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
    +      thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
    +        .setTime_stamp(System.currentTimeMillis)
    +      CarbonEnv.get.carbonMetastore
    +        .updateTableSchema(carbonTable.getCarbonTableIdentifier,
    --- End diff --
   
    Write common functions, avoid duplicate code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106093450
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala ---
    @@ -304,38 +341,76 @@ class CarbonMetastore(conf: RuntimeConfig, val storePath: String) {
         if (tableExists(tableName, Some(dbName))(sparkSession)) {
           sys.error(s"Table [$tableName] already exists under Database [$dbName]")
         }
    +    val schemaEvolutionEntry = new SchemaEvolutionEntry(tableInfo.getLastUpdatedTime)
         val schemaConverter = new ThriftWrapperSchemaConverterImpl
         val thriftTableInfo = schemaConverter
           .fromWrapperToExternalTableInfo(tableInfo, dbName, tableName)
    -    val schemaEvolutionEntry = new SchemaEvolutionEntry(tableInfo.getLastUpdatedTime)
         thriftTableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history
           .add(schemaEvolutionEntry)
    +    val carbonTablePath = createSchemaThriftFile(tableInfo,
    +      thriftTableInfo,
    +      dbName,
    +      tableName)(sparkSession)
    +    updateSchemasUpdatedTime(touchSchemaFileSystemTime(dbName, tableName))
    +    LOGGER.info(s"Table $tableName for Database $dbName created successfully.")
    +    carbonTablePath
    +  }
     
    +  /**
    +   * This method will write the schema thrift file in carbon store and load table metadata
    +   *
    +   * @param tableInfo
    +   * @param thriftTableInfo
    +   * @param dbName
    +   * @param tableName
    +   * @param sparkSession
    +   * @return
    +   */
    +  private def createSchemaThriftFile(
    +      tableInfo: org.apache.carbondata.core.metadata.schema.table.TableInfo,
    +      thriftTableInfo: org.apache.carbondata.format.TableInfo,
    +      dbName: String, tableName: String)
    +    (sparkSession: SparkSession): String = {
         val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName,
           tableInfo.getFactTable.getTableId)
         val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
         val schemaFilePath = carbonTablePath.getSchemaFilePath
         val schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath)
         tableInfo.setMetaDataFilepath(schemaMetadataPath)
         tableInfo.setStorePath(storePath)
    -    CarbonMetadata.getInstance().loadTableMetadata(tableInfo)
    -    val tableMeta = new TableMeta(carbonTableIdentifier, storePath,
    -      CarbonMetadata.getInstance().getCarbonTable(dbName + "_" + tableName))
    -
         val fileType = FileFactory.getFileType(schemaMetadataPath)
         if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
           FileFactory.mkdirs(schemaMetadataPath, fileType)
         }
         val thriftWriter = new ThriftWriter(schemaFilePath, false)
    -    thriftWriter.open()
    +    thriftWriter.open(FileWriteOperation.OVERWRITE)
         thriftWriter.write(thriftTableInfo)
         thriftWriter.close()
    +    removeTableFromMetadata(dbName, tableName)
    --- End diff --
   
    directly call refresh flow after updating alter schema


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106095253
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -136,6 +140,298 @@ case class AlterTableCompaction(alterTableModel: AlterTableModel) extends Runnab
       }
     }
     
    +private[sql] case class AlterTableDataTypeChange(
    +    alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel) extends RunnableCommand {
    +
    +  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
    +
    +  def run(sparkSession: SparkSession): Seq[Row] = {
    +    val tableName = alterTableDataTypeChangeModel.tableName
    +    val dbName = alterTableDataTypeChangeModel.databaseName
    +      .getOrElse(sparkSession.catalog.currentDatabase)
    +    LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
    +    val relation =
    +      CarbonEnv.get.carbonMetastore
    +        .lookupRelation(Option(dbName), tableName)(sparkSession)
    +        .asInstanceOf[CarbonRelation]
    +    if (relation == null) {
    +      LOGGER.audit(s"Alter table change data type request has failed. " +
    +                   s"Table $dbName.$tableName does not exist")
    +      sys.error(s"Table $dbName.$tableName does not exist")
    +    }
    +    // acquire the lock first
    +    val table = relation.tableMeta.carbonTable
    +    val carbonLock = CarbonLockFactory
    +      .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
    +        LockUsage.METADATA_LOCK)
    +    try {
    +      // get the latest carbon table and check for column existence
    +      val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
    +      val columnName = alterTableDataTypeChangeModel.columnName
    +      var carbonColumnToBeModified: CarbonColumn = null
    +      val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala
    +      // read the latest schema file
    +      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
    +        carbonTable.getCarbonTableIdentifier)
    +      val tableMetadataFile = carbonTablePath.getSchemaFilePath
    +      val tableInfo: org.apache.carbondata.format.TableInfo = CarbonEnv.get.carbonMetastore
    +        .readSchemaFile(tableMetadataFile)
    +      // maintain the added column for schema evolution history
    +      var addColumnSchema: org.apache.carbondata.format.ColumnSchema = null
    +      var deletedColumnSchema: org.apache.carbondata.format.ColumnSchema = null
    +      val columnSchemaList = tableInfo.fact_table.table_columns.asScala
    +      columnSchemaList.foreach { columnSchema =>
    +        if (columnSchema.column_name.equalsIgnoreCase(columnName)) {
    +          deletedColumnSchema = CarbonScalaUtil.createColumnSchemaCopyObject(columnSchema)
    +          columnSchema.setData_type(DataTypeConverterUtil
    +            .convertToThriftDataType(alterTableDataTypeChangeModel.dataTypeInfo.dataType))
    +          columnSchema.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
    +          columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
    +          addColumnSchema = columnSchema
    +        }
    +      }
    +      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +      schemaEvolutionEntry.setAdded(List(addColumnSchema).asJava)
    +      schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava)
    +      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
    +        .setTime_stamp(System.currentTimeMillis)
    +      CarbonEnv.get.carbonMetastore
    +        .updateTableSchema(carbonTable.getCarbonTableIdentifier,
    +          tableInfo,
    +          schemaEvolutionEntry,
    +          carbonTable.getStorePath)(sparkSession)
    +
    +      val tableIdentifier = TableIdentifier(tableName, Some(dbName))
    +      val schema = CarbonEnv.get.carbonMetastore
    +        .lookupRelation(tableIdentifier)(sparkSession).schema.json
    +      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.runSqlHive(
    +        s"ALTER TABLE $dbName.$tableName SET TBLPROPERTIES('spark.sql.sources.schema'='$schema')")
    +      sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
    +      LOGGER.info(s"Alter table for data type change is successful for table $dbName.$tableName")
    +      LOGGER.audit(s"Alter table for data type change is successful for table $dbName.$tableName")
    +    } catch {
    +      case e: Exception =>
    +        LOGGER.error("Alter table change datatype failed : " + e.getMessage)
    +        throw e
    +    } finally {
    +      // release lock after command execution completion
    +      if (carbonLock != null) {
    +        if (carbonLock.unlock()) {
    +          LOGGER.info("Alter table change data type lock released successfully")
    +        } else {
    +          LOGGER.error("Unable to release lock during alter table change data type operation")
    +        }
    +      }
    +    }
    +    Seq.empty
    +  }
    +}
    +
    +private[sql] case class AlterTableAddColumns(
    +    alterTableAddColumnsModel: AlterTableAddColumnsModel) extends RunnableCommand {
    +
    +  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
    +
    +  def run(sparkSession: SparkSession): Seq[Row] = {
    +    val tableName = alterTableAddColumnsModel.tableName
    +    val dbName = alterTableAddColumnsModel.databaseName
    +      .getOrElse(sparkSession.catalog.currentDatabase)
    +    LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
    +    val relation =
    +      CarbonEnv.get.carbonMetastore
    +        .lookupRelation(Option(dbName), tableName)(sparkSession)
    +        .asInstanceOf[CarbonRelation]
    +    if (relation == null) {
    +      LOGGER.audit(s"Alter table add columns request has failed. " +
    +                   s"Table $dbName.$tableName does not exist")
    +      sys.error(s"Table $dbName.$tableName does not exist")
    +    }
    +    // acquire the lock first
    +    val table = relation.tableMeta.carbonTable
    +    val carbonLock = CarbonLockFactory
    +      .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
    +        LockUsage.METADATA_LOCK)
    +    try {
    +      // get the latest carbon table and check for column existence
    +      val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
    +      // read the latest schema file
    +      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
    +        carbonTable.getCarbonTableIdentifier)
    +      val tableMetadataFile = carbonTablePath.getSchemaFilePath
    +      val thriftTableInfo: org.apache.carbondata.format.TableInfo = CarbonEnv.get.carbonMetastore
    +        .readSchemaFile(tableMetadataFile)
    +      val schemaConverter = new ThriftWrapperSchemaConverterImpl()
    +      val wrapperTableInfo = schemaConverter
    +        .fromExternalToWrapperTableInfo(thriftTableInfo,
    +          dbName,
    +          tableName,
    +          carbonTable.getStorePath)
    +      val newCols = new AlterTableProcessor(alterTableAddColumnsModel,
    +        dbName,
    +        wrapperTableInfo,
    +        carbonTablePath,
    +        carbonTable.getCarbonTableIdentifier,
    +        carbonTable.getStorePath).process
    +      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata
    +      .schema.SchemaEvolutionEntry()
    +      schemaEvolutionEntry.setTimeStamp(System.currentTimeMillis)
    +      schemaEvolutionEntry.setAdded(newCols.toList.asJava)
    +
    +      val thriftTable = schemaConverter
    +        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
    +      thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
    +        .setTime_stamp(System.currentTimeMillis)
    +      CarbonEnv.get.carbonMetastore
    +        .updateTableSchema(carbonTable.getCarbonTableIdentifier,
    +          thriftTable,
    +          schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
    +          carbonTable.getStorePath)(sparkSession)
    +
    +      val tableIdentifier = TableIdentifier(tableName, Some(dbName))
    +      val schema = CarbonEnv.get.carbonMetastore
    +        .lookupRelation(tableIdentifier)(sparkSession).schema.json
    +      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.runSqlHive(
    +        s"ALTER TABLE $dbName.$tableName SET TBLPROPERTIES('spark.sql.sources.schema'='$schema')")
    +      sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
    +      LOGGER.info(s"Alter table for add columns is successful for table $dbName.$tableName")
    +      LOGGER.audit(s"Alter table for add columns is successful for table $dbName.$tableName")
    +    } catch {
    +      case e: Exception =>
    +        LOGGER.error("Alter table add columns failed : " + e.getMessage)
    +        throw e
    +    } finally {
    +      // release lock after command execution completion
    +      if (carbonLock != null) {
    +        if (carbonLock.unlock()) {
    +          LOGGER.info("Alter table add columns lock released successfully")
    +        } else {
    +          LOGGER.error("Unable to release lock during alter table add columns operation")
    +        }
    +      }
    +    }
    +    Seq.empty
    +  }
    +}
    +
    +private[sql] case class AlterTableDropColumns(
    +    alterTableDropColumnModel: AlterTableDropColumnModel) extends RunnableCommand {
    +
    +  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
    +
    +  def run(sparkSession: SparkSession): Seq[Row] = {
    +    val tableName = alterTableDropColumnModel.tableName
    +    val dbName = alterTableDropColumnModel.databaseName
    +      .getOrElse(sparkSession.catalog.currentDatabase)
    +    LOGGER.audit(s"Alter table drop columns request has been received for $dbName.$tableName")
    +    val relation =
    +      CarbonEnv.get.carbonMetastore
    +        .lookupRelation(Option(dbName), tableName)(sparkSession)
    +        .asInstanceOf[CarbonRelation]
    +    if (relation == null) {
    +      LOGGER.audit(s"Alter table drop columns request has failed. " +
    +                   s"Table $dbName.$tableName does not exist")
    +      sys.error(s"Table $dbName.$tableName does not exist")
    +    }
    +    // acquire the lock first
    +    val table = relation.tableMeta.carbonTable
    +    val carbonLock = CarbonLockFactory
    +      .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
    +        LockUsage.METADATA_LOCK)
    +    try {
    +      // get the latest carbon table and check for column existence
    +      val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
    +      // check each column existence in the table
    +      val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
    +      var dictionaryColumns = ListBuffer[CarbonColumn]()
    +      var keyColumnCountToBeDeleted = 0
    +      // TODO: if deleted column list includes shared dictionary/bucketted column throw an error
    +      alterTableDropColumnModel.columns.foreach { column =>
    +        var columnExist = false
    +        tableColumns.foreach { tableColumn =>
    +          // column should not be already deleted and should exist in the table
    +          if (!tableColumn.isInvisible && column.equalsIgnoreCase(tableColumn.getColName)) {
    +            if (tableColumn.isDimesion) {
    +              keyColumnCountToBeDeleted += 1
    +              if (tableColumn.hasEncoding(Encoding.DICTIONARY)) {
    +                dictionaryColumns += tableColumn
    +              }
    +            }
    +            columnExist = true
    +          }
    +        }
    +        if (!columnExist) {
    +          sys.error(s"Column $column does not exists in the table $dbName.$tableName")
    +        }
    +      }
    +      // take the total key column count. key column to be deleted should not
    +      // be >= key columns in schema
    +      var totalKeyColumnInSchema = 0
    +      tableColumns.foreach { tableColumn =>
    +        // column should not be already deleted and should exist in the table
    +        if (!tableColumn.isInvisible && tableColumn.isDimesion) {
    +          totalKeyColumnInSchema += 1
    +        }
    +      }
    +      if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
    +        sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
    +      }
    +      // read the latest schema file
    +      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
    +        carbonTable.getCarbonTableIdentifier)
    +      val tableMetadataFile = carbonTablePath.getSchemaFilePath
    +      val tableInfo: org.apache.carbondata.format.TableInfo = CarbonEnv.get.carbonMetastore
    +        .readSchemaFile(tableMetadataFile)
    +      // maintain the deleted columns for schema evolution history
    +      var deletedColumnSchema = ListBuffer[org.apache.carbondata.format.ColumnSchema]()
    +      val columnSchemaList = tableInfo.fact_table.table_columns.asScala
    +      alterTableDropColumnModel.columns.foreach { column =>
    +        columnSchemaList.foreach { columnSchema =>
    +          if (!columnSchema.invisible && column.equalsIgnoreCase(columnSchema.column_name)) {
    +            deletedColumnSchema += CarbonScalaUtil.createColumnSchemaCopyObject(columnSchema)
    +            columnSchema.invisible = true
    +          }
    +        }
    +      }
    +      // add deleted columns to schema evolution history and update the schema
    +      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
    +        .setTime_stamp(System.currentTimeMillis)
    +      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
    +      schemaEvolutionEntry.setRemoved(deletedColumnSchema.toList.asJava)
    --- End diff --
   
    Add function addEvolutionEntry


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #641: [CARBONDATA-767] Alter table support for ca...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/641
 
    Build Failed  with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1147/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user nareshpr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106112761
 
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala ---
    @@ -77,4 +77,40 @@ object DataTypeConverterUtil {
           case DataType.STRUCT => "struct"
         }
       }
    +
    +  /**
    +   * convert from wrapper to external data type
    +   *
    +   * @param dataType
    +   * @return
    +   */
    +  def convertToThriftDataType(dataType: String): org.apache.carbondata.format.DataType = {
    --- End diff --
   
    There is no direct string to thrift datatype conversion. When creating table, we first convert string to Wrapper datatype and from wrapper datatype, we convert into thrift datatype. This method is required for alter table, as we r directly converting string to thrift type in alter table change datatype


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user nareshpr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106112796
 
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala ---
    @@ -194,4 +196,102 @@ object CarbonScalaUtil {
           }
         }
       }
    +
    +  /**
    +   * This method will validate a column for its data type and check whether the column data type
    +   * can be modified and update if conditions are met
    +   *
    +   * @param dataTypeInfo
    +   * @param carbonColumn
    +   */
    +  def validateColumnDataType(dataTypeInfo: DataTypeInfo, carbonColumn: CarbonColumn): Unit = {
    +    carbonColumn.getDataType.getName match {
    +      case "INT" =>
    +        if (!dataTypeInfo.dataType.equals("bigint")) {
    +          sys
    +            .error(s"Given column ${ carbonColumn.getColName } with data type ${
    +              carbonColumn
    +                .getDataType.getName
    +            } cannot be modified. Int can only be changed to bigInt")
    +        }
    +      case "DECIMAL" =>
    +        if (!dataTypeInfo.dataType.equals("decimal")) {
    +          sys
    +            .error(s"Given column ${ carbonColumn.getColName } with data type ${
    +              carbonColumn.getDataType.getName
    +            } cannot be modified. Decimal can be only be changed to Decimal of higher precision")
    +        }
    +        if (dataTypeInfo.precision <= carbonColumn.getColumnSchema.getPrecision) {
    +          sys
    +            .error(s"Given column ${
    +              carbonColumn
    +                .getColName
    +            } cannot be modified. Specified precision value ${
    +              dataTypeInfo
    +                .precision
    +            } should be greater or equal to current precision value ${
    +              carbonColumn.getColumnSchema
    +                .getPrecision
    +            }")
    +        } else if (dataTypeInfo.scale <= carbonColumn.getColumnSchema.getScale) {
    +          sys
    +            .error(s"Given column ${
    +              carbonColumn
    +                .getColName
    +            } cannot be modified. Specified scale value ${
    +              dataTypeInfo
    +                .scale
    +            } should be greater or equal to current scale value ${
    +              carbonColumn.getColumnSchema
    +                .getScale
    +            }")
    +        } else {
    +          // difference of precision and scale specified by user should not be less than the
    +          // difference of already existing precision and scale else it will result in data loss
    +          val carbonColumnPrecisionScaleDiff = carbonColumn.getColumnSchema.getPrecision -
    +                                               carbonColumn.getColumnSchema.getScale
    +          val dataInfoPrecisionScaleDiff = dataTypeInfo.precision - dataTypeInfo.scale
    +          if (dataInfoPrecisionScaleDiff < carbonColumnPrecisionScaleDiff) {
    +            sys
    +              .error(s"Given column ${
    +                carbonColumn
    +                  .getColName
    +              } cannot be modified. Specified precision and scale values will lead to data loss")
    +          }
    +        }
    +      case _ =>
    +        sys
    +          .error(s"Given column ${ carbonColumn.getColName } with data type ${
    +            carbonColumn
    +              .getDataType.getName
    +          } cannot be modified. Only Int and Decimal data types are allowed for modification")
    +    }
    +  }
    +
    +  /**
    +   * This method will create a copy of the same object
    +   *
    +   * @param thriftColumnSchema object to be cloned
    +   * @return
    +   */
    +  def createColumnSchemaCopyObject(thriftColumnSchema: org.apache.carbondata.format.ColumnSchema)
    --- End diff --
   
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user nareshpr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106125641
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala ---
    @@ -304,38 +341,76 @@ class CarbonMetastore(conf: RuntimeConfig, val storePath: String) {
         if (tableExists(tableName, Some(dbName))(sparkSession)) {
           sys.error(s"Table [$tableName] already exists under Database [$dbName]")
         }
    +    val schemaEvolutionEntry = new SchemaEvolutionEntry(tableInfo.getLastUpdatedTime)
         val schemaConverter = new ThriftWrapperSchemaConverterImpl
         val thriftTableInfo = schemaConverter
           .fromWrapperToExternalTableInfo(tableInfo, dbName, tableName)
    -    val schemaEvolutionEntry = new SchemaEvolutionEntry(tableInfo.getLastUpdatedTime)
         thriftTableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history
           .add(schemaEvolutionEntry)
    +    val carbonTablePath = createSchemaThriftFile(tableInfo,
    +      thriftTableInfo,
    +      dbName,
    +      tableName)(sparkSession)
    +    updateSchemasUpdatedTime(touchSchemaFileSystemTime(dbName, tableName))
    +    LOGGER.info(s"Table $tableName for Database $dbName created successfully.")
    +    carbonTablePath
    +  }
     
    +  /**
    +   * This method will write the schema thrift file in carbon store and load table metadata
    +   *
    +   * @param tableInfo
    +   * @param thriftTableInfo
    +   * @param dbName
    +   * @param tableName
    +   * @param sparkSession
    +   * @return
    +   */
    +  private def createSchemaThriftFile(
    +      tableInfo: org.apache.carbondata.core.metadata.schema.table.TableInfo,
    +      thriftTableInfo: org.apache.carbondata.format.TableInfo,
    +      dbName: String, tableName: String)
    +    (sparkSession: SparkSession): String = {
         val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName,
           tableInfo.getFactTable.getTableId)
         val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
         val schemaFilePath = carbonTablePath.getSchemaFilePath
         val schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath)
         tableInfo.setMetaDataFilepath(schemaMetadataPath)
         tableInfo.setStorePath(storePath)
    -    CarbonMetadata.getInstance().loadTableMetadata(tableInfo)
    -    val tableMeta = new TableMeta(carbonTableIdentifier, storePath,
    -      CarbonMetadata.getInstance().getCarbonTable(dbName + "_" + tableName))
    -
         val fileType = FileFactory.getFileType(schemaMetadataPath)
         if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
           FileFactory.mkdirs(schemaMetadataPath, fileType)
         }
         val thriftWriter = new ThriftWriter(schemaFilePath, false)
    -    thriftWriter.open()
    +    thriftWriter.open(FileWriteOperation.OVERWRITE)
         thriftWriter.write(thriftTableInfo)
         thriftWriter.close()
    +    removeTableFromMetadata(dbName, tableName)
    --- End diff --
   
    As per old code, when new table is created, table info is added into CarbonMetastore and modified.mdt file is timestamp is updated to refresh in other sessions. Those changes are extracted to a new method and used in CreateTable and AlterTable flow, updating mdt file code is missing, which i added


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user nareshpr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106129635
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala ---
    @@ -129,4 +134,80 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
           case databaseName ~ tableName ~ limit =>
             ShowLoadsCommand(convertDbNameToLowerCase(databaseName), tableName.toLowerCase(), limit)
         }
    +
    +  protected lazy val alterTableModifyDataType: Parser[LogicalPlan] =
    +    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ CHANGE ~ ident ~ ident ~
    +    ident ~ opt("(" ~> rep1sep(valueOptions, ",") <~ ")") <~ opt(";") ^^ {
    +      case dbName ~ table ~ change ~ columnName ~ columnNameCopy ~ dataType ~ values =>
    +        // both the column names should be same
    +        CommonUtil.validateColumnNames(columnName, columnNameCopy)
    --- End diff --
   
    Ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user nareshpr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106131227
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala ---
    @@ -129,4 +134,80 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
           case databaseName ~ tableName ~ limit =>
             ShowLoadsCommand(convertDbNameToLowerCase(databaseName), tableName.toLowerCase(), limit)
         }
    +
    +  protected lazy val alterTableModifyDataType: Parser[LogicalPlan] =
    +    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ CHANGE ~ ident ~ ident ~
    +    ident ~ opt("(" ~> rep1sep(valueOptions, ",") <~ ")") <~ opt(";") ^^ {
    +      case dbName ~ table ~ change ~ columnName ~ columnNameCopy ~ dataType ~ values =>
    +        // both the column names should be same
    +        CommonUtil.validateColumnNames(columnName, columnNameCopy)
    +        val alterTableChangeDataTypeModel =
    +          AlterTableDataTypeChangeModel(parseDataType(dataType.toLowerCase, values),
    +            convertDbNameToLowerCase(dbName),
    +            table.toLowerCase,
    +            columnName.toLowerCase,
    +            columnNameCopy.toLowerCase)
    +        AlterTableDataTypeChange(alterTableChangeDataTypeModel)
    +    }
    +
    +  protected lazy val alterTableAddColumns: Parser[LogicalPlan] =
    +    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~
    +    (ADD ~> COLUMNS ~> "(" ~> repsep(anyFieldDef, ",") <~ ")") ~
    +    (TBLPROPERTIES ~> "(" ~> repsep(loadOptions, ",") <~ ")").? <~ opt(";") ^^ {
    +      case dbName ~ table ~ fields ~ tblProp =>
    +        fields.foreach{ f =>
    +          if (isComplexDimDictionaryExclude(f.dataType.get)) {
    +            throw new MalformedCarbonCommandException(
    +              s"Add column is unsupported for complex datatype column: ${f.column}")
    +          }
    +        }
    +        val tableProps = if (tblProp.isDefined) {
    +          // default value should not be converted to lower case
    +          val tblProps = tblProp.get.map(f => if (f._1.toLowerCase.startsWith("default.value.")) {
    +            f._1 -> f._2
    +          } else {
    +            f._1 -> f._2.toLowerCase
    +          })
    +          scala.collection.mutable.Map(tblProps: _*)
    +        } else {
    +          scala.collection.mutable.Map.empty[String, String]
    +        }
    +
    +        val tableModel = prepareTableModel (false,
    +          convertDbNameToLowerCase(dbName),
    +          table.toLowerCase,
    +          fields.map(convertFieldNamesToLowercase),
    +          Seq.empty,
    +          tableProps,
    +          None,
    +          true)
    +
    +        val alterTableAddColumnsModel = AlterTableAddColumnsModel(convertDbNameToLowerCase(dbName),
    +          table,
    +          tableProps,
    +          tableModel.dimCols,
    +          tableModel.msrCols,
    +          tableModel.highcardinalitydims.getOrElse(Seq.empty))
    +        AlterTableAddColumns(alterTableAddColumnsModel)
    +    }
    +
    +  private def convertFieldNamesToLowercase(field: Field): Field = {
    +    val name = field.column.toLowerCase
    +    field.copy(column = name, name = Some(name))
    +  }
    +  protected lazy val alterTableDropColumn: Parser[LogicalPlan] =
    --- End diff --
   
    Ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #641: [CARBONDATA-767] Alter table support for ca...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/641
 
    Build Failed  with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1154/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #641: [CARBONDATA-767] Alter table support for ca...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user manishgupta88 commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/641
 
    @gvramana ...Handled all review comments and all the test cases are passing. Please refer the below link.
    kindly review and merge


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #641: [CARBONDATA-767] Alter table support for ca...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/641
 
    Build Failed  with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1156/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #641: [CARBONDATA-767] Alter table support for ca...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/641
 
    Build Failed  with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1160/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #641: [CARBONDATA-767] Alter table support for ca...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user nareshpr commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/641
 
    @gvramana ...Handled all review comments and all the test cases are passing. Please refer the below link.
    http://136.243.101.176:8080/job/ManualApacheCarbonPRBuilder2.1/103/
    kindly review and merge


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #641: [CARBONDATA-767] Alter table support for ca...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/641
 
    Build Failed  with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1162/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106174923
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -313,6 +307,100 @@ private[sql] case class AlterTableAddColumns(
       }
     }
     
    +private[sql] case class AlterTableRenameTable(alterTableRenameModel: AlterTableRenameModel)
    --- End diff --
   
    Move these commands and case class AlterTableRenameTable to AlterTableCommands.scala


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106189693
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -747,14 +835,14 @@ case class LoadTable(
                 true
               } else {
                 LOGGER.error("Can't use single_pass, because SINGLE_PASS and ALL_DICTIONARY_PATH" +
    -              "can not be used together, and USE_KETTLE must be set as false")
    +                         "can not be used together, and USE_KETTLE must be set as false")
    --- End diff --
   
    wrong indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #641: [CARBONDATA-767] Alter table support...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user nareshpr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/641#discussion_r106192438
 
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -313,6 +307,100 @@ private[sql] case class AlterTableAddColumns(
       }
     }
     
    +private[sql] case class AlterTableRenameTable(alterTableRenameModel: AlterTableRenameModel)
    --- End diff --
   
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
1234