You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/02/10 14:07:18 UTC

[GitHub] [carbondata] akashrn5 opened a new pull request #3608: [WIP]Si feature

akashrn5 opened a new pull request #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608
 
 
    ### Why is this PR needed?
    
    
    ### What changes were proposed in this PR?
   
       
    ### Does this PR introduce any user interface change?
    - No
    - Yes. (please explain the change and update document)
   
    ### Is any new testcase added?
    - No
    - Yes
   
       
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584593717
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/236/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378748901
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
 ##########
 @@ -61,7 +61,8 @@
 
   public DiskBasedDMSchemaStorageProvider(String storePath) {
     this.storePath = CarbonUtil.checkAndAppendHDFSUrl(storePath);
-    this.mdtFilePath = storePath + CarbonCommonConstants.FILE_SEPARATOR + "datamap.mdtfile";
+    this.mdtFilePath = CarbonUtil.checkAndAppendHDFSUrl(
 
 Review comment:
   this.mdtFilePath = this.storePath + CarbonCommonConstants.FILE_SEPARATOR + "datamap.mdtfile";

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585167971
 
 
   > please add description
   
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378796482
 
 

 ##########
 File path: integration/spark2/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
 ##########
 @@ -0,0 +1,574 @@
+/*
+ * 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.secondaryindex.command
+
+import java.io.IOException
+import java.util
+import java.util.UUID
+
+import scala.collection.JavaConverters._
+import scala.language.implicitConversions
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.execution.command.DataCommand
+import org.apache.spark.sql.hive.{CarbonHiveMetadataUtil, CarbonRelation}
+import org.apache.spark.sql.secondaryindex.exception.IndexTableExistException
+import org.apache.spark.sql.secondaryindex.load.CarbonInternalLoaderUtil
+import org.apache.spark.sql.secondaryindex.util.{CarbonInternalScalaUtil, IndexTableUtil}
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolutionEntry, SchemaReader}
+import org.apache.carbondata.core.metadata.schema.indextable.{IndexMetadata, IndexTableInfo}
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
+import org.apache.carbondata.core.service.impl.ColumnUniqueIdGenerator
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.events.{CreateTablePostExecutionEvent, CreateTablePreExecutionEvent, OperationContext, OperationListenerBus}
+
+class ErrorMessage(message: String) extends Exception(message) {
+}
+
+ /**
+  * Command for index table creation
+  * @param indexModel      SecondaryIndex model holding the index infomation
+  * @param tableProperties SI table properties
+  * @param isCreateSIndex  if false then will not create index table schema in the carbonstore
+   *                        and will avoid dataload for SI creation.
+  */
+ private[sql] case class CreateIndexTable(indexModel: SecondaryIndex,
+     tableProperties: scala.collection.mutable.Map[String, String],
+     var isCreateSIndex: Boolean = true)
+   extends DataCommand {
+
+   val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+   override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val databaseName = CarbonEnv.getDatabaseName(indexModel.databaseName)(sparkSession)
+    indexModel.databaseName = Some(databaseName)
+    val tableName = indexModel.tableName
+    val storePath = CarbonProperties.getStorePath
+    val dbLocation = CarbonEnv.getDatabaseLocation(databaseName, sparkSession)
+    val indexTableName = indexModel.indexTableName
+
+    val tablePath = dbLocation + CarbonCommonConstants.FILE_SEPARATOR + indexTableName
+     setAuditTable(databaseName, indexTableName)
+     setAuditInfo(Map("Column names" -> indexModel.columnNames.toString(),
+       "Parent TableName" -> indexModel.tableName,
+       "SI Table Properties" -> tableProperties.toString()))
+    LOGGER.info(
+      s"Creating Index with Database name [$databaseName] and Index name [$indexTableName]")
+    val catalog = CarbonEnv.getInstance(sparkSession).carbonMetaStore
+    val identifier = TableIdentifier(tableName, indexModel.databaseName)
+    var carbonTable: CarbonTable = null
+    var locks: List[ICarbonLock] = List()
+    var oldIndexInfo = ""
+
+    try {
+      carbonTable = CarbonEnv.getCarbonTable(indexModel.databaseName, tableName)(sparkSession)
+      if (carbonTable == null) {
+        throw new ErrorMessage(s"Parent Table $databaseName.$tableName is not found")
+      }
+
+      if (carbonTable != null &&
+          (carbonTable.isFileLevelFormat || !carbonTable.getTableInfo.isTransactionalTable)) {
+        throw new MalformedCarbonCommandException(
+          "Unsupported operation on non transactional table")
+      }
+
+      if (carbonTable.isStreamingSink) {
+        throw new ErrorMessage(
+          s"Parent Table  ${ carbonTable.getDatabaseName }." +
+          s"${ carbonTable.getTableName }" +
+          s" is Streaming Table and Secondary index on Streaming table is not supported ")
+      }
+
+      if (carbonTable.isHivePartitionTable) {
+        throw new ErrorMessage(
+          s"Parent Table  ${ carbonTable.getDatabaseName }." +
+          s"${ carbonTable.getTableName }" +
+          s" is Partition Table and Secondary index on Partition table is not supported ")
 
 Review comment:
   yes, there will some issues in pruning part, but we can support

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585810276
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1981/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378752394
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
 ##########
 @@ -621,6 +624,52 @@ public boolean accept(CarbonFile file) {
     }
   }
 
+  /**
+   * this is the clean up added specifically for SI table, because after we merge the data files
+   * inside the secondary index table, we need to delete the stale carbondata files.
+   * refer {@link org.apache.spark.sql.secondaryindex.rdd.CarbonSIRebuildRDD}
+   */
+  private static void cleanUpDataFilesAfterSmallFIlesMergeForSI(CarbonTable table,
 
 Review comment:
   cleanUpDataFilesAfterSmallFIlesMergeForSI => cleanUpDataFilesAfterSmallFilesMergeForSI

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378309556
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
 ##########
 @@ -113,8 +113,9 @@ public BlockletDataMapIndexWrapper get(TableBlockIndexUniqueIdentifierWrapper id
           BlockDataMap blockletDataMap =
               loadAndGetDataMap(identifier, indexFileStore, blockMetaInfoMap,
                   identifierWrapper.getCarbonTable(),
-                  identifierWrapper.isAddTableBlockToUnsafeAndLRUCache(),
-                  identifierWrapper.getConfiguration(), indexInfos);
+                  identifierWrapper.isAddToUnsafe(),
+                  identifierWrapper.getConfiguration(),
+                  identifierWrapper.isSerializeDmStore(), indexInfos);
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378265942
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
 ##########
 @@ -46,7 +48,17 @@ public void finishWriting() {
     // do nothing in default implementation
   }
 
+  public void serializeMemoryBlock() {
+  }
+
+  public void copyToMemoryBlock() {
 
 Review comment:
   why empty implementation? not abstract?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378811867
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
 ##########
 @@ -61,7 +61,8 @@
 
   public DiskBasedDMSchemaStorageProvider(String storePath) {
     this.storePath = CarbonUtil.checkAndAppendHDFSUrl(storePath);
-    this.mdtFilePath = storePath + CarbonCommonConstants.FILE_SEPARATOR + "datamap.mdtfile";
+    this.mdtFilePath = CarbonUtil.checkAndAppendHDFSUrl(
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585720917
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/275/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378769971
 
 

 ##########
 File path: integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -609,6 +613,137 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         CarbonAlterTableDropColumnCommand(alterTableDropColumnModel)
     }
 
+  protected lazy val indexCommands: Parser[LogicalPlan] =
+    showIndexes | createIndexTable | dropIndexTable | registerIndexes | rebuildIndex
+
+  protected lazy val createIndexTable: Parser[LogicalPlan] =
+    CREATE ~> INDEX ~> ident ~ (ON ~> TABLE ~> (ident <~ ".").? ~ ident) ~
+    ("(" ~> repsep(ident, ",") <~ ")") ~ (AS ~> stringLit) ~
+    (TBLPROPERTIES ~> "(" ~> repsep(options, ",") <~ ")").? <~ opt(";") ^^ {
+      case indexTableName ~ table ~ cols ~ indexStoreType ~ tblProp =>
+
+        if (!("carbondata".equalsIgnoreCase(indexStoreType) ||
+              "org.apache.carbondata.format".equalsIgnoreCase(indexStoreType))) {
+          sys.error("Not a carbon format request")
+        }
+
+        val (dbName, tableName) = table match {
+          case databaseName ~ tableName => (databaseName, tableName.toLowerCase())
+        }
+
+        val tableProperties = if (tblProp.isDefined) {
+          val tblProps = tblProp.get.map(f => f._1 -> f._2)
+          scala.collection.mutable.Map(tblProps: _*)
+        } else {
+          scala.collection.mutable.Map.empty[String, String]
+        }
+        // validate the tableBlockSize from table properties
+        CommonUtil.validateSize(tableProperties, CarbonCommonConstants.TABLE_BLOCKSIZE)
+        // validate for supported table properties
+        validateTableProperties(tableProperties)
+        // validate column_meta_cache proeperty if defined
+        val tableColumns: List[String] = cols.map(f => f.toLowerCase)
+        validateColumnMetaCacheAndCacheLevelProeprties(dbName,
+          indexTableName.toLowerCase,
+          tableColumns,
+          tableProperties)
+        validateColumnCompressorProperty(tableProperties
+          .getOrElse(CarbonCommonConstants.COMPRESSOR, null))
+        val indexTableModel = SecondaryIndex(dbName,
+          tableName.toLowerCase,
+          tableColumns,
+          indexTableName.toLowerCase)
+        CreateIndexTable(indexTableModel, tableProperties)
+    }
+
+  private def validateColumnMetaCacheAndCacheLevelProeprties(dbName: Option[String],
+      tableName: String,
+      tableColumns: Seq[String],
+      tableProperties: scala.collection.mutable.Map[String, String]): Unit = {
+    // validate column_meta_cache property
+    if (tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
+      CommonUtil.validateColumnMetaCacheFields(
+        dbName.getOrElse(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+        tableName,
+        tableColumns,
+        tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).get,
+        tableProperties)
+    }
+    // validate cache_level property
+    if (tableProperties.get(CarbonCommonConstants.CACHE_LEVEL).isDefined) {
+      CommonUtil.validateCacheLevel(
+        tableProperties.get(CarbonCommonConstants.CACHE_LEVEL).get,
+        tableProperties)
+    }
+  }
+
+  private def validateColumnCompressorProperty(columnCompressor: String): Unit = {
+    // Add validatation for column compressor when creating index table
+    try {
+      if (null != columnCompressor) {
+        CompressorFactory.getInstance().getCompressor(columnCompressor)
+      }
+    } catch {
+      case ex: UnsupportedOperationException =>
+        throw new InvalidConfigurationException(ex.getMessage)
+    }
+  }
+
+  /**
+   * this method validates if index table properties contains other than supported ones
+   *
+   * @param tableProperties
+   */
+  private def validateTableProperties(tableProperties: scala.collection.mutable.Map[String,
+    String]) = {
+    val supportedPropertiesForIndexTable = Seq("TABLE_BLOCKSIZE",
+      "COLUMN_META_CACHE",
+      "CACHE_LEVEL",
+      CarbonCommonConstants.COMPRESSOR.toUpperCase)
+    tableProperties.foreach { property =>
+      if (!supportedPropertiesForIndexTable.contains(property._1.toUpperCase)) {
+        val errorMessage = "Unsupported Table property in index creation: " + property._1.toString
+        throw new MalformedCarbonCommandException(errorMessage)
+      }
+    }
+  }
+
+  protected lazy val dropIndexTable: Parser[LogicalPlan] =
+    DROP ~> INDEX ~> opt(IF ~> EXISTS) ~ ident ~ (ON ~> (ident <~ ".").? ~ ident) <~ opt(";") ^^ {
+      case ifexist ~ indexTableName ~ table =>
+        val (dbName, tableName) = table match {
+          case databaseName ~ tableName => (databaseName, tableName.toLowerCase())
+        }
+        DropIndexCommand(ifexist.isDefined, dbName, indexTableName.toLowerCase, tableName)
+    }
+
+  protected lazy val showIndexes: Parser[LogicalPlan] =
+    (SHOW ~> opt(FORMATTED)) ~> (INDEXES | INDEX) ~> ON ~> ident ~ opt((FROM | IN) ~> ident) <~
 
 Review comment:
   how about 
   `(SHOW  ~> INDEXES ~> ON ~> (ident <~ ".").? ~ ident`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584172876
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1916/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585220935
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/265/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378265311
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
 ##########
 @@ -32,14 +32,14 @@
  * It is the wrapper around datamap and related filter expression. By using it user can apply
  * datamaps in expression style.
  */
-public interface DataMapExprWrapper extends Serializable {
+public abstract class DataMapExprWrapper implements Serializable {
 
 Review comment:
   you can still use interface, java 8 interface can have default implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584523384
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1931/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378309350
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2341,4 +2347,78 @@ private CarbonCommonConstants() {
    * Default first day of week
    */
   public static final String CARBON_TIMESERIES_FIRST_DAY_OF_WEEK_DEFAULT = "SUNDAY";
+
+  @CarbonProperty
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER =
+      "carbon.infilter.subquery.pushdown.enable";
+
+
+  /**
+   * CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT
+   */
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT = "false";
+
+  /**
+   * key to get broadcast record size from properties
+   */
+  @CarbonProperty
+  public static final String BROADCAST_RECORD_SIZE = "broadcast.record.size";
+
+  /**
+   * default broadcast record size
+   */
+  public static final String DEFAULT_BROADCAST_RECORD_SIZE = "100";
+
+  /**
+   * to enable SI lookup partial string
+   */
+  @CarbonProperty
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING = "carbon.si.lookup.partialstring";
+
+  /**
+   * default value of ENABLE_SI_LOOKUP_PARTIALSTRING
+   */
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING_DEFAULT = "true";
+
+  /**
+   * configuration for launching the number of threads during secondary index creation
+   */
+  @CarbonProperty
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS =
+      "carbon.secondary.index.creation.threads";
+
+  /**
+   * default value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS_DEFAULT = "1";
+
+  /**
+   * max value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final int CARBON_SECONDARY_INDEX_CREATION_THREADS_MAX = 50;
+
+  /**
+   * threshold of high cardinality
+   */
+  @CarbonProperty
+  public static final String HIGH_CARDINALITY_THRESHOLD = "high.cardinality.threshold";
 
 Review comment:
   yes, removed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378811931
 
 

 ##########
 File path: integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -609,6 +613,137 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         CarbonAlterTableDropColumnCommand(alterTableDropColumnModel)
     }
 
+  protected lazy val indexCommands: Parser[LogicalPlan] =
+    showIndexes | createIndexTable | dropIndexTable | registerIndexes | rebuildIndex
+
+  protected lazy val createIndexTable: Parser[LogicalPlan] =
+    CREATE ~> INDEX ~> ident ~ (ON ~> TABLE ~> (ident <~ ".").? ~ ident) ~
+    ("(" ~> repsep(ident, ",") <~ ")") ~ (AS ~> stringLit) ~
+    (TBLPROPERTIES ~> "(" ~> repsep(options, ",") <~ ")").? <~ opt(";") ^^ {
+      case indexTableName ~ table ~ cols ~ indexStoreType ~ tblProp =>
+
+        if (!("carbondata".equalsIgnoreCase(indexStoreType) ||
+              "org.apache.carbondata.format".equalsIgnoreCase(indexStoreType))) {
+          sys.error("Not a carbon format request")
+        }
+
+        val (dbName, tableName) = table match {
+          case databaseName ~ tableName => (databaseName, tableName.toLowerCase())
+        }
+
+        val tableProperties = if (tblProp.isDefined) {
+          val tblProps = tblProp.get.map(f => f._1 -> f._2)
+          scala.collection.mutable.Map(tblProps: _*)
+        } else {
+          scala.collection.mutable.Map.empty[String, String]
+        }
+        // validate the tableBlockSize from table properties
+        CommonUtil.validateSize(tableProperties, CarbonCommonConstants.TABLE_BLOCKSIZE)
+        // validate for supported table properties
+        validateTableProperties(tableProperties)
+        // validate column_meta_cache proeperty if defined
+        val tableColumns: List[String] = cols.map(f => f.toLowerCase)
+        validateColumnMetaCacheAndCacheLevelProeprties(dbName,
+          indexTableName.toLowerCase,
+          tableColumns,
+          tableProperties)
+        validateColumnCompressorProperty(tableProperties
+          .getOrElse(CarbonCommonConstants.COMPRESSOR, null))
+        val indexTableModel = SecondaryIndex(dbName,
+          tableName.toLowerCase,
+          tableColumns,
+          indexTableName.toLowerCase)
+        CreateIndexTable(indexTableModel, tableProperties)
+    }
+
+  private def validateColumnMetaCacheAndCacheLevelProeprties(dbName: Option[String],
+      tableName: String,
+      tableColumns: Seq[String],
+      tableProperties: scala.collection.mutable.Map[String, String]): Unit = {
+    // validate column_meta_cache property
+    if (tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
+      CommonUtil.validateColumnMetaCacheFields(
+        dbName.getOrElse(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+        tableName,
+        tableColumns,
+        tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).get,
+        tableProperties)
+    }
+    // validate cache_level property
+    if (tableProperties.get(CarbonCommonConstants.CACHE_LEVEL).isDefined) {
+      CommonUtil.validateCacheLevel(
+        tableProperties.get(CarbonCommonConstants.CACHE_LEVEL).get,
+        tableProperties)
+    }
+  }
+
+  private def validateColumnCompressorProperty(columnCompressor: String): Unit = {
+    // Add validatation for column compressor when creating index table
+    try {
+      if (null != columnCompressor) {
+        CompressorFactory.getInstance().getCompressor(columnCompressor)
+      }
+    } catch {
+      case ex: UnsupportedOperationException =>
+        throw new InvalidConfigurationException(ex.getMessage)
+    }
+  }
+
+  /**
+   * this method validates if index table properties contains other than supported ones
+   *
+   * @param tableProperties
+   */
+  private def validateTableProperties(tableProperties: scala.collection.mutable.Map[String,
+    String]) = {
+    val supportedPropertiesForIndexTable = Seq("TABLE_BLOCKSIZE",
+      "COLUMN_META_CACHE",
+      "CACHE_LEVEL",
+      CarbonCommonConstants.COMPRESSOR.toUpperCase)
+    tableProperties.foreach { property =>
+      if (!supportedPropertiesForIndexTable.contains(property._1.toUpperCase)) {
+        val errorMessage = "Unsupported Table property in index creation: " + property._1.toString
+        throw new MalformedCarbonCommandException(errorMessage)
+      }
+    }
+  }
+
+  protected lazy val dropIndexTable: Parser[LogicalPlan] =
+    DROP ~> INDEX ~> opt(IF ~> EXISTS) ~ ident ~ (ON ~> (ident <~ ".").? ~ ident) <~ opt(";") ^^ {
+      case ifexist ~ indexTableName ~ table =>
+        val (dbName, tableName) = table match {
+          case databaseName ~ tableName => (databaseName, tableName.toLowerCase())
+        }
+        DropIndexCommand(ifexist.isDefined, dbName, indexTableName.toLowerCase, tableName)
+    }
+
+  protected lazy val showIndexes: Parser[LogicalPlan] =
+    (SHOW ~> opt(FORMATTED)) ~> (INDEXES | INDEX) ~> ON ~> ident ~ opt((FROM | IN) ~> ident) <~
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584216533
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/221/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378309460
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
 ##########
 @@ -526,4 +532,29 @@ public long getRowCount(List<Segment> allsegments, final List<PartitionSpec> par
     return totalRowCount;
   }
 
+  /**
+   * Method to prune the segments based on task min/max values
+   *
+   * @param segments
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584624780
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1939/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378261576
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2341,4 +2347,78 @@ private CarbonCommonConstants() {
    * Default first day of week
    */
   public static final String CARBON_TIMESERIES_FIRST_DAY_OF_WEEK_DEFAULT = "SUNDAY";
+
+  @CarbonProperty
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER =
+      "carbon.infilter.subquery.pushdown.enable";
+
+
+  /**
+   * CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT
+   */
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT = "false";
+
+  /**
+   * key to get broadcast record size from properties
+   */
+  @CarbonProperty
+  public static final String BROADCAST_RECORD_SIZE = "broadcast.record.size";
+
+  /**
+   * default broadcast record size
+   */
+  public static final String DEFAULT_BROADCAST_RECORD_SIZE = "100";
+
+  /**
+   * to enable SI lookup partial string
+   */
+  @CarbonProperty
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING = "carbon.si.lookup.partialstring";
+
+  /**
+   * default value of ENABLE_SI_LOOKUP_PARTIALSTRING
+   */
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING_DEFAULT = "true";
+
+  /**
+   * configuration for launching the number of threads during secondary index creation
+   */
+  @CarbonProperty
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS =
+      "carbon.secondary.index.creation.threads";
+
+  /**
+   * default value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS_DEFAULT = "1";
+
+  /**
+   * max value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final int CARBON_SECONDARY_INDEX_CREATION_THREADS_MAX = 50;
+
+  /**
+   * threshold of high cardinality
+   */
+  @CarbonProperty
+  public static final String HIGH_CARDINALITY_THRESHOLD = "high.cardinality.threshold";
 
 Review comment:
   I think this is not required now

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378811841
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2341,4 +2347,72 @@ private CarbonCommonConstants() {
    * Default first day of week
    */
   public static final String CARBON_TIMESERIES_FIRST_DAY_OF_WEEK_DEFAULT = "SUNDAY";
+
+  @CarbonProperty
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER =
+      "carbon.infilter.subquery.pushdown.enable";
+
+
+  /**
+   * CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT
+   */
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT = "false";
+
+  /**
+   * key to get broadcast record size from properties
+   */
+  @CarbonProperty
+  public static final String BROADCAST_RECORD_SIZE = "broadcast.record.size";
+
+  /**
+   * default broadcast record size
+   */
+  public static final String DEFAULT_BROADCAST_RECORD_SIZE = "100";
+
+  /**
+   * to enable SI lookup partial string
+   */
+  @CarbonProperty
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING = "carbon.si.lookup.partialstring";
+
+  /**
+   * default value of ENABLE_SI_LOOKUP_PARTIALSTRING
+   */
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING_DEFAULT = "true";
+
+  /**
+   * configuration for launching the number of threads during secondary index creation
+   */
+  @CarbonProperty
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS =
+      "carbon.secondary.index.creation.threads";
+
+  /**
+   * default value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS_DEFAULT = "1";
+
+  /**
+   * max value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final int CARBON_SECONDARY_INDEX_CREATION_THREADS_MAX = 50;
+
+  /**
+   * Enable SI segment Compaction / merge small files
+   */
+  @CarbonProperty
+  public static final String CARBON_SI_SEGMENT_MERGE = "carbon.si.segment.merge";
+
+  /**
+   * Default value for SI segment Compaction / merge small files
+   * Making this true degrade the LOAD performance
+   * When the number of small files increase for SI segments(it can happen as number of columns will
+   * be less and we store position id and reference columns), user an either set to true which will
+   * merge the data files for upcoming loads or run SI rebuild command which does this job for all
+   * segments. (REBUILD INDEX <index_table>)
+   */
+  public static final String DEFAULT_CARBON_SI_SEGMENT_MERGE = "false";
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584792902
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/244/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378264017
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
 ##########
 @@ -526,4 +532,29 @@ public long getRowCount(List<Segment> allsegments, final List<PartitionSpec> par
     return totalRowCount;
   }
 
+  /**
+   * Method to prune the segments based on task min/max values
+   *
+   * @param segments
 
 Review comment:
   remove it if you are not writing description

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584589294
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/235/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378265942
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
 ##########
 @@ -46,7 +48,17 @@ public void finishWriting() {
     // do nothing in default implementation
   }
 
+  public void serializeMemoryBlock() {
+  }
+
+  public void copyToMemoryBlock() {
 
 Review comment:
   why empty implementation?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
QiangCai commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585124824
 
 
   please add description

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584821209
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1946/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584594254
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1938/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378262296
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2341,4 +2347,78 @@ private CarbonCommonConstants() {
    * Default first day of week
    */
   public static final String CARBON_TIMESERIES_FIRST_DAY_OF_WEEK_DEFAULT = "SUNDAY";
+
+  @CarbonProperty
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER =
 
 Review comment:
   Please explain in comment when should user set to true

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584604417
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/237/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584247400
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1923/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] asfgit closed pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378309533
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
 ##########
 @@ -46,7 +48,17 @@ public void finishWriting() {
     // do nothing in default implementation
   }
 
+  public void serializeMemoryBlock() {
+  }
+
+  public void copyToMemoryBlock() {
 
 Review comment:
   since only for unsafe implementation, its required.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585146097
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1964/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378309510
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java
 ##########
 @@ -32,14 +32,14 @@
  * It is the wrapper around datamap and related filter expression. By using it user can apply
  * datamaps in expression style.
  */
-public interface DataMapExprWrapper extends Serializable {
+public abstract class DataMapExprWrapper implements Serializable {
 
 Review comment:
   since some users are still using older java version,i think we can keep until we completely move out.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378309605
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
 ##########
 @@ -133,8 +134,9 @@ public BlockletDataMapIndexWrapper get(TableBlockIndexUniqueIdentifierWrapper id
               BlockDataMap blockletDataMap =
                   loadAndGetDataMap(blockIndexUniqueIdentifier, indexFileStore, blockMetaInfoMap,
                       identifierWrapper.getCarbonTable(),
-                      identifierWrapper.isAddTableBlockToUnsafeAndLRUCache(),
-                      identifierWrapper.getConfiguration(), indexInfos);
+                      identifierWrapper.isAddToUnsafe(),
+                      identifierWrapper.getConfiguration(),
+                      identifierWrapper.isSerializeDmStore(), indexInfos);
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585814883
 
 
   LGTM

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585087486
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1951/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378768681
 
 

 ##########
 File path: integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -609,6 +613,137 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         CarbonAlterTableDropColumnCommand(alterTableDropColumnModel)
     }
 
+  protected lazy val indexCommands: Parser[LogicalPlan] =
+    showIndexes | createIndexTable | dropIndexTable | registerIndexes | rebuildIndex
+
+  protected lazy val createIndexTable: Parser[LogicalPlan] =
+    CREATE ~> INDEX ~> ident ~ (ON ~> TABLE ~> (ident <~ ".").? ~ ident) ~
+    ("(" ~> repsep(ident, ",") <~ ")") ~ (AS ~> stringLit) ~
+    (TBLPROPERTIES ~> "(" ~> repsep(options, ",") <~ ")").? <~ opt(";") ^^ {
+      case indexTableName ~ table ~ cols ~ indexStoreType ~ tblProp =>
+
+        if (!("carbondata".equalsIgnoreCase(indexStoreType) ||
+              "org.apache.carbondata.format".equalsIgnoreCase(indexStoreType))) {
 
 Review comment:
   support only  "carbondata"

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585180711
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/264/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378261883
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2341,4 +2347,78 @@ private CarbonCommonConstants() {
    * Default first day of week
    */
   public static final String CARBON_TIMESERIES_FIRST_DAY_OF_WEEK_DEFAULT = "SUNDAY";
+
+  @CarbonProperty
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER =
+      "carbon.infilter.subquery.pushdown.enable";
+
+
+  /**
+   * CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT
+   */
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT = "false";
+
+  /**
+   * key to get broadcast record size from properties
+   */
+  @CarbonProperty
+  public static final String BROADCAST_RECORD_SIZE = "broadcast.record.size";
+
+  /**
+   * default broadcast record size
+   */
+  public static final String DEFAULT_BROADCAST_RECORD_SIZE = "100";
+
+  /**
+   * to enable SI lookup partial string
+   */
+  @CarbonProperty
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING = "carbon.si.lookup.partialstring";
+
+  /**
+   * default value of ENABLE_SI_LOOKUP_PARTIALSTRING
+   */
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING_DEFAULT = "true";
+
+  /**
+   * configuration for launching the number of threads during secondary index creation
+   */
+  @CarbonProperty
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS =
+      "carbon.secondary.index.creation.threads";
+
+  /**
+   * default value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS_DEFAULT = "1";
+
+  /**
+   * max value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final int CARBON_SECONDARY_INDEX_CREATION_THREADS_MAX = 50;
+
+  /**
+   * threshold of high cardinality
+   */
+  @CarbonProperty
+  public static final String HIGH_CARDINALITY_THRESHOLD = "high.cardinality.threshold";
+
+  public static final String HIGH_CARDINALITY_THRESHOLD_DEFAULT = "1000000";
+
+  public static final int HIGH_CARDINALITY_THRESHOLD_MIN = 10000;
+
+  /**
+   * Enable SI segment Compaction / merge small files
+   */
+  @CarbonProperty
+  public static final String CARBON_SI_SEGMENT_MERGE = "carbon.si.segment.merge";
+
+  /**
+   * Default value for SI segment Compaction / merge small files
+   * Making this true degrade the LOAD performance
 
 Review comment:
   please explain in comment when should user set to true?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584504158
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/229/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378309399
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2341,4 +2347,78 @@ private CarbonCommonConstants() {
    * Default first day of week
    */
   public static final String CARBON_TIMESERIES_FIRST_DAY_OF_WEEK_DEFAULT = "SUNDAY";
+
+  @CarbonProperty
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER =
+      "carbon.infilter.subquery.pushdown.enable";
+
+
+  /**
+   * CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT
+   */
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT = "false";
+
+  /**
+   * key to get broadcast record size from properties
+   */
+  @CarbonProperty
+  public static final String BROADCAST_RECORD_SIZE = "broadcast.record.size";
+
+  /**
+   * default broadcast record size
+   */
+  public static final String DEFAULT_BROADCAST_RECORD_SIZE = "100";
+
+  /**
+   * to enable SI lookup partial string
+   */
+  @CarbonProperty
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING = "carbon.si.lookup.partialstring";
+
+  /**
+   * default value of ENABLE_SI_LOOKUP_PARTIALSTRING
+   */
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING_DEFAULT = "true";
+
+  /**
+   * configuration for launching the number of threads during secondary index creation
+   */
+  @CarbonProperty
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS =
+      "carbon.secondary.index.creation.threads";
+
+  /**
+   * default value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS_DEFAULT = "1";
+
+  /**
+   * max value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final int CARBON_SECONDARY_INDEX_CREATION_THREADS_MAX = 50;
+
+  /**
+   * threshold of high cardinality
+   */
+  @CarbonProperty
+  public static final String HIGH_CARDINALITY_THRESHOLD = "high.cardinality.threshold";
+
+  public static final String HIGH_CARDINALITY_THRESHOLD_DEFAULT = "1000000";
+
+  public static final int HIGH_CARDINALITY_THRESHOLD_MIN = 10000;
+
+  /**
+   * Enable SI segment Compaction / merge small files
+   */
+  @CarbonProperty
+  public static final String CARBON_SI_SEGMENT_MERGE = "carbon.si.segment.merge";
+
+  /**
+   * Default value for SI segment Compaction / merge small files
+   * Making this true degrade the LOAD performance
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378771821
 
 

 ##########
 File path: integration/spark2/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
 ##########
 @@ -0,0 +1,574 @@
+/*
+ * 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.secondaryindex.command
+
+import java.io.IOException
+import java.util
+import java.util.UUID
+
+import scala.collection.JavaConverters._
+import scala.language.implicitConversions
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.execution.command.DataCommand
+import org.apache.spark.sql.hive.{CarbonHiveMetadataUtil, CarbonRelation}
+import org.apache.spark.sql.secondaryindex.exception.IndexTableExistException
+import org.apache.spark.sql.secondaryindex.load.CarbonInternalLoaderUtil
+import org.apache.spark.sql.secondaryindex.util.{CarbonInternalScalaUtil, IndexTableUtil}
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolutionEntry, SchemaReader}
+import org.apache.carbondata.core.metadata.schema.indextable.{IndexMetadata, IndexTableInfo}
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
+import org.apache.carbondata.core.service.impl.ColumnUniqueIdGenerator
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.events.{CreateTablePostExecutionEvent, CreateTablePreExecutionEvent, OperationContext, OperationListenerBus}
+
+class ErrorMessage(message: String) extends Exception(message) {
+}
+
+ /**
+  * Command for index table creation
+  * @param indexModel      SecondaryIndex model holding the index infomation
+  * @param tableProperties SI table properties
+  * @param isCreateSIndex  if false then will not create index table schema in the carbonstore
+   *                        and will avoid dataload for SI creation.
+  */
+ private[sql] case class CreateIndexTable(indexModel: SecondaryIndex,
+     tableProperties: scala.collection.mutable.Map[String, String],
+     var isCreateSIndex: Boolean = true)
+   extends DataCommand {
+
+   val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+   override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val databaseName = CarbonEnv.getDatabaseName(indexModel.databaseName)(sparkSession)
+    indexModel.databaseName = Some(databaseName)
+    val tableName = indexModel.tableName
+    val storePath = CarbonProperties.getStorePath
+    val dbLocation = CarbonEnv.getDatabaseLocation(databaseName, sparkSession)
+    val indexTableName = indexModel.indexTableName
+
+    val tablePath = dbLocation + CarbonCommonConstants.FILE_SEPARATOR + indexTableName
+     setAuditTable(databaseName, indexTableName)
+     setAuditInfo(Map("Column names" -> indexModel.columnNames.toString(),
+       "Parent TableName" -> indexModel.tableName,
+       "SI Table Properties" -> tableProperties.toString()))
+    LOGGER.info(
+      s"Creating Index with Database name [$databaseName] and Index name [$indexTableName]")
+    val catalog = CarbonEnv.getInstance(sparkSession).carbonMetaStore
+    val identifier = TableIdentifier(tableName, indexModel.databaseName)
+    var carbonTable: CarbonTable = null
+    var locks: List[ICarbonLock] = List()
+    var oldIndexInfo = ""
+
+    try {
+      carbonTable = CarbonEnv.getCarbonTable(indexModel.databaseName, tableName)(sparkSession)
+      if (carbonTable == null) {
+        throw new ErrorMessage(s"Parent Table $databaseName.$tableName is not found")
+      }
+
+      if (carbonTable != null &&
+          (carbonTable.isFileLevelFormat || !carbonTable.getTableInfo.isTransactionalTable)) {
+        throw new MalformedCarbonCommandException(
+          "Unsupported operation on non transactional table")
+      }
+
+      if (carbonTable.isStreamingSink) {
+        throw new ErrorMessage(
+          s"Parent Table  ${ carbonTable.getDatabaseName }." +
+          s"${ carbonTable.getTableName }" +
+          s" is Streaming Table and Secondary index on Streaming table is not supported ")
+      }
+
+      if (carbonTable.isHivePartitionTable) {
+        throw new ErrorMessage(
+          s"Parent Table  ${ carbonTable.getDatabaseName }." +
+          s"${ carbonTable.getTableName }" +
+          s" is Partition Table and Secondary index on Partition table is not supported ")
 
 Review comment:
   is it easy to support it in the future?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585143772
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/261/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378266500
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
 ##########
 @@ -113,8 +113,9 @@ public BlockletDataMapIndexWrapper get(TableBlockIndexUniqueIdentifierWrapper id
           BlockDataMap blockletDataMap =
               loadAndGetDataMap(identifier, indexFileStore, blockMetaInfoMap,
                   identifierWrapper.getCarbonTable(),
-                  identifierWrapper.isAddTableBlockToUnsafeAndLRUCache(),
-                  identifierWrapper.getConfiguration(), indexInfos);
+                  identifierWrapper.isAddToUnsafe(),
+                  identifierWrapper.getConfiguration(),
+                  identifierWrapper.isSerializeDmStore(), indexInfos);
 
 Review comment:
   move indexInfos to next line

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3608: [CARBONDATA-3680]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378266561
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
 ##########
 @@ -133,8 +134,9 @@ public BlockletDataMapIndexWrapper get(TableBlockIndexUniqueIdentifierWrapper id
               BlockDataMap blockletDataMap =
                   loadAndGetDataMap(blockIndexUniqueIdentifier, indexFileStore, blockMetaInfoMap,
                       identifierWrapper.getCarbonTable(),
-                      identifierWrapper.isAddTableBlockToUnsafeAndLRUCache(),
-                      identifierWrapper.getConfiguration(), indexInfos);
+                      identifierWrapper.isAddToUnsafe(),
+                      identifierWrapper.getConfiguration(),
+                      identifierWrapper.isSerializeDmStore(), indexInfos);
 
 Review comment:
   move indexInfos to next line

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378811891
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
 ##########
 @@ -621,6 +624,52 @@ public boolean accept(CarbonFile file) {
     }
   }
 
+  /**
+   * this is the clean up added specifically for SI table, because after we merge the data files
+   * inside the secondary index table, we need to delete the stale carbondata files.
+   * refer {@link org.apache.spark.sql.secondaryindex.rdd.CarbonSIRebuildRDD}
+   */
+  private static void cleanUpDataFilesAfterSmallFIlesMergeForSI(CarbonTable table,
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585100396
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/252/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378744023
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2341,4 +2347,72 @@ private CarbonCommonConstants() {
    * Default first day of week
    */
   public static final String CARBON_TIMESERIES_FIRST_DAY_OF_WEEK_DEFAULT = "SUNDAY";
+
+  @CarbonProperty
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER =
+      "carbon.infilter.subquery.pushdown.enable";
+
+
+  /**
+   * CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT
+   */
+  public static final String CARBON_PUSH_LEFTSEMIEXIST_JOIN_AS_IN_FILTER_DEFAULT = "false";
+
+  /**
+   * key to get broadcast record size from properties
+   */
+  @CarbonProperty
+  public static final String BROADCAST_RECORD_SIZE = "broadcast.record.size";
+
+  /**
+   * default broadcast record size
+   */
+  public static final String DEFAULT_BROADCAST_RECORD_SIZE = "100";
+
+  /**
+   * to enable SI lookup partial string
+   */
+  @CarbonProperty
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING = "carbon.si.lookup.partialstring";
+
+  /**
+   * default value of ENABLE_SI_LOOKUP_PARTIALSTRING
+   */
+  public static final String ENABLE_SI_LOOKUP_PARTIALSTRING_DEFAULT = "true";
+
+  /**
+   * configuration for launching the number of threads during secondary index creation
+   */
+  @CarbonProperty
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS =
+      "carbon.secondary.index.creation.threads";
+
+  /**
+   * default value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final String CARBON_SECONDARY_INDEX_CREATION_THREADS_DEFAULT = "1";
+
+  /**
+   * max value configuration for launching the number of threads during secondary
+   * index creation
+   */
+  public static final int CARBON_SECONDARY_INDEX_CREATION_THREADS_MAX = 50;
+
+  /**
+   * Enable SI segment Compaction / merge small files
+   */
+  @CarbonProperty
+  public static final String CARBON_SI_SEGMENT_MERGE = "carbon.si.segment.merge";
+
+  /**
+   * Default value for SI segment Compaction / merge small files
+   * Making this true degrade the LOAD performance
+   * When the number of small files increase for SI segments(it can happen as number of columns will
+   * be less and we store position id and reference columns), user an either set to true which will
+   * merge the data files for upcoming loads or run SI rebuild command which does this job for all
+   * segments. (REBUILD INDEX <index_table>)
+   */
+  public static final String DEFAULT_CARBON_SI_SEGMENT_MERGE = "false";
 
 Review comment:
   DEFAULT_CARBON_SI_SEGMENT_MERGE => CARBON_SI_SEGMENT_MERGE_DEFAULT

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378811905
 
 

 ##########
 File path: integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -609,6 +613,137 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         CarbonAlterTableDropColumnCommand(alterTableDropColumnModel)
     }
 
+  protected lazy val indexCommands: Parser[LogicalPlan] =
+    showIndexes | createIndexTable | dropIndexTable | registerIndexes | rebuildIndex
+
+  protected lazy val createIndexTable: Parser[LogicalPlan] =
+    CREATE ~> INDEX ~> ident ~ (ON ~> TABLE ~> (ident <~ ".").? ~ ident) ~
+    ("(" ~> repsep(ident, ",") <~ ")") ~ (AS ~> stringLit) ~
+    (TBLPROPERTIES ~> "(" ~> repsep(options, ",") <~ ")").? <~ opt(";") ^^ {
+      case indexTableName ~ table ~ cols ~ indexStoreType ~ tblProp =>
+
+        if (!("carbondata".equalsIgnoreCase(indexStoreType) ||
+              "org.apache.carbondata.format".equalsIgnoreCase(indexStoreType))) {
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378796921
 
 

 ##########
 File path: secondary_index/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 ##########
 @@ -0,0 +1,17 @@
+## ------------------------------------------------------------------------
+## 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.
+## ------------------------------------------------------------------------
+org.apache.spark.sql.CarbonSource
 
 Review comment:
   removed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585748604
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1978/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585260154
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/268/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585768043
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/278/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585128056
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1955/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585068177
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/248/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#discussion_r378774558
 
 

 ##########
 File path: secondary_index/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 ##########
 @@ -0,0 +1,17 @@
+## ------------------------------------------------------------------------
+## 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.
+## ------------------------------------------------------------------------
+org.apache.spark.sql.CarbonSource
 
 Review comment:
   already exists in integration/spark2 module 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-584144623
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/214/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on issue #3608: [WIP]Si feature

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on issue #3608: [WIP]Si feature
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585090987
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3608: [CARBONDATA-3680][alpha-feature]Support Secondary Index feature on carbon table.
URL: https://github.com/apache/carbondata/pull/3608#issuecomment-585304077
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/1971/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services