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/03/30 14:02:09 UTC

[GitHub] [carbondata] Indhumathi27 opened a new pull request #3688: [WIP] Refactor Index Metadata

Indhumathi27 opened a new pull request #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688
 
 
   This PR depends on [PR-3661](https://github.com/apache/carbondata/pull/3661) 
   
   ### 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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406012114
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala
 ##########
 @@ -192,6 +226,42 @@ object CarbonInternalMetastore {
           LOGGER.error(e.getMessage)
       }
     }
+    if (null != indexExists) {
+      if (null != carbonTable && (null == indexExists || indexExists.toBoolean)) {
 
 Review comment:
   indexExists is not null, that is when it enters this 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] Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406071579
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/ShowIndexesCommand.scala
 ##########
 @@ -54,69 +62,87 @@ case class ShowIndexesCommand(
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     val carbonTable = CarbonEnv.getCarbonTable(dbNameOp, tableName)(sparkSession)
     setAuditTable(carbonTable)
-    getFileIndexInfo(carbonTable) ++ getSIInfo(sparkSession, carbonTable)
-  }
-
-  // get info for 'index datamap'
-  private def getFileIndexInfo(carbonTable: CarbonTable): Seq[Row] = {
-    val indexes = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable).asScala
-    if (indexes != null && indexes.nonEmpty) {
-      indexes.map { index =>
-        Row(
-          index.getDataMapName,
-          index.getProviderName,
-          index.getIndexColumns.mkString(","),
-          index.getPropertiesAsString,
-          index.getStatus.name(),
-          index.getSyncStatus
-        )
-      }
-    } else {
-      Seq.empty
-    }
+    getIndexInfo(sparkSession, carbonTable)
   }
 
-  // get info for SI
-  private def getSIInfo(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
+  private def getIndexInfo(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
     CarbonInternalMetastore.refreshIndexInfo(
       carbonTable.getDatabaseName, tableName, carbonTable)(sparkSession)
-    val indexesMap = CarbonInternalScalaUtil.getIndexesMap(carbonTable)
-    if (null == indexesMap) {
-      throw new Exception("Secondary index information is not loaded in main table")
-    }
-    val indexTableMap = indexesMap.asScala
-    if (indexTableMap.nonEmpty) {
-      val indexList = indexTableMap.map { indexInfo =>
-        try {
-          val isSITableEnabled = sparkSession.sessionState.catalog
-            .getTableMetadata(TableIdentifier(indexInfo._1, dbNameOp)).storage.properties
-            .getOrElse("isSITableEnabled", "true").equalsIgnoreCase("true")
-          if (isSITableEnabled) {
-            (indexInfo._1, indexInfo._2.asScala.mkString(","), "enabled")
-          } else {
-            (indexInfo._1, indexInfo._2.asScala.mkString(","), "disabled")
+    val indexesMap = CarbonIndexUtil.getIndexesMap(carbonTable)
+    if (null != indexesMap) {
+      val indexTableMap = indexesMap.asScala
+      if (indexTableMap.nonEmpty) {
+        val secondaryIndex = indexTableMap.get(CarbonIndexProvider.SI.getIndexProviderName)
+        var finalIndexList: Seq[(String, String, String, String, String, String)] = Seq.empty
+
+        if (secondaryIndex.isDefined && null != secondaryIndex.get) {
+          val siIterator = secondaryIndex.get.entrySet().iterator()
+          while (siIterator.hasNext) {
+            val indexInfo = siIterator.next()
+            try {
+              val isSITableEnabled = sparkSession.sessionState.catalog
+                .getTableMetadata(TableIdentifier(indexInfo.getKey, dbNameOp)).storage.properties
+                .getOrElse("isSITableEnabled", "true").equalsIgnoreCase("true")
+              if (isSITableEnabled) {
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants.INDEX_COLUMNS), "NA", "enabled", "NA")
+              } else {
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants
+                                     .INDEX_COLUMNS), "NA", "disabled", "NA")
+              }
+            } catch {
+              case ex: Exception =>
+                LOGGER.error(s"Access storage properties from hive failed for index table: ${
+                  indexInfo.getKey
+                }")
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants.INDEX_COLUMNS), "NA", "UNKNOWN", "NA")
+            }
+          }
+        }
+
+        indexesMap.asScala
+          .filter(map => !map._1.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName))
 
 Review comment:
   Yes. But if we keep single map with provider and index info, it will be easier to validate index, add index info to hive and drop index. Else, seperately, we need to do for both

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405994284
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -101,49 +100,29 @@ private DataMapStoreManager() {
   }
 
   /**
-   * It only gives the visible datamaps
-   */
-  List<TableIndex> getAllVisibleIndexes(CarbonTable carbonTable) throws IOException {
-    CarbonSessionInfo sessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo();
-    List<TableIndex> allDataMaps = getAllIndexes(carbonTable);
-    Iterator<TableIndex> dataMapIterator = allDataMaps.iterator();
-    while (dataMapIterator.hasNext()) {
-      TableIndex dataMap = dataMapIterator.next();
-      String dbName = carbonTable.getDatabaseName();
-      String tableName = carbonTable.getTableName();
-      String dmName = dataMap.getDataMapSchema().getDataMapName();
-      // TODO: need support get the visible status of datamap without sessionInfo in the future
-      if (sessionInfo != null) {
-        boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
-            String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
-                dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
-        if (!isDmVisible) {
-          LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
-              dmName, dbName, tableName));
-          dataMapIterator.remove();
-        }
-      } else {
-        String message = "Carbon session info is null";
-        LOGGER.info(message);
-      }
-    }
-    return allDataMaps;
-  }
-
-  /**
-   * It gives all indexes except the default index.
+   * It gives all indexes except the default index and secondary index.
+   * Collect's Coarse grain and Fine grain indexes on a table
    *
    * @return
    */
   public List<TableIndex> getAllIndexes(CarbonTable carbonTable) throws IOException {
-    List<DataMapSchema> dataMapSchemas = getDataMapSchemasOfTable(carbonTable);
+    String indexMeta = carbonTable.getTableInfo().getFactTable().getTableProperties()
+        .get(carbonTable.getCarbonTableIdentifier().getTableId());
+    IndexMetadata indexMetadata = IndexMetadata.deserialize(indexMeta);
     List<TableIndex> indexes = new ArrayList<>();
-    if (dataMapSchemas != null) {
-      for (DataMapSchema dataMapSchema : dataMapSchemas) {
-        RelationIdentifier identifier = dataMapSchema.getParentTables().get(0);
-        if (dataMapSchema.isIndex() && identifier.getTableId()
-            .equals(carbonTable.getTableId())) {
-          indexes.add(getIndex(carbonTable, dataMapSchema));
+    if (null != indexMetadata) {
+      // get bloom indexes and lucene indexes
+      for (Map.Entry<String, Map<String, Map<String, String>>> providerEntry : indexMetadata
+          .getIndexesMap().entrySet()) {
+        for (Map.Entry<String, Map<String, String>> indexEntry : providerEntry.getValue()
+            .entrySet()) {
+          if (!indexEntry.getValue().get(CarbonCommonConstants.INDEX_PROVIDER)
 
 Review comment:
   In CarbonTable, we are adding indexInfo with Provider. In SI Flow, only  indexes having provider name as "SI" will be sent. Still need to unify the flow? If yes, can you please give some inputs

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611470184
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2691/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607776793
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/911/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611958770
 
 
   @ajantha-bhat Fixed all comments. Please review

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406576369
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/index/IndexStoreManager.java
 ##########
 @@ -820,8 +715,11 @@ public synchronized void clearInvalidIndex(CarbonTable carbonTable, List<String>
   }
 
   private boolean hasCGIndex(CarbonTable carbonTable) throws IOException {
-    for (TableIndex tableIndex : getAllVisibleIndexes(carbonTable)) {
-      if (tableIndex.getIndexFactory().getDataMapLevel().equals(IndexLevel.CG)) {
+    if (null == carbonTable) {
 
 Review comment:
   It will be null in case of file format flow

----------------------------------------------------------------
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] ajantha-bhat commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-615098233
 
 
   @Indhumathi27 : It is better to remove system folder disk based schema storage code. If we are not using it anymore.

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


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-608427778
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/919/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-606107531
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/886/
   

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406576086
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonRefreshIndexCommand.scala
 ##########
 @@ -71,30 +80,81 @@ case class CarbonRefreshIndexCommand(
   private def refreshIndex(
       sparkSession: SparkSession,
       parentTable: CarbonTable,
-      indexOp: Optional[DataMapSchema]): Unit = {
-    val schema = indexOp.get
+      indexMetaData: IndexMetadata): Unit = {
+    var indexInfo: util.Map[String, String] = new util.HashMap[String, String]()
+    val allIndexesIterator = indexMetaData.getIndexesMap.entrySet().iterator()
+    breakable {
+      while (allIndexesIterator.hasNext) {
+        val currentIndex = allIndexesIterator.next()
+        if (!currentIndex.getKey.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName)) {
+          val indexIterator = currentIndex.getValue.entrySet().iterator()
+          while (indexIterator.hasNext) {
+            val indexEntry = indexIterator.next()
+            if (indexEntry.getKey.equalsIgnoreCase(indexName)) {
 
 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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612822372
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1011/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-606550851
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/888/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-613988575
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2748/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-613987310
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1035/
   

----------------------------------------------------------------
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] ajantha-bhat edited a comment on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat edited a comment on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-610773012
 
 
   @Indhumathi27 : Just checked few files on high level. Couldn't focus on functionality as there is distracting 300 file rename changes. Suggest to separate rename and functionality(multi-tenant) PR

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405986131
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -1202,21 +1192,64 @@ public boolean isIndexTable() throws IOException {
     }
   }
 
+  public List<String> getIndexTableNames(String indexProvider) throws IOException {
+    deserializeIndexMetadata();
+    if (null != indexMetadata) {
+      return indexMetadata.getIndexTables(indexProvider);
+    } else {
+      return new ArrayList<>();
+    }
+  }
+
   public String getIndexInfo() throws IOException {
+    return getIndexInfo(null);
+  }
+
+  public IndexMetadata getIndexMetadata() throws IOException {
+    deserializeIndexMetadata();
+    return indexMetadata;
+  }
+
+  public String getIndexInfo(String indexProvider) throws IOException {
     deserializeIndexMetadata();
     if (null != indexMetadata) {
-      IndexTableInfo[] indexTableInfos =
-          new IndexTableInfo[indexMetadata.getIndexesMap().entrySet().size()];
-      int index = 0;
-      if (!isIndexTable()) {
-        for (Map.Entry<String, List<String>> entry : indexMetadata.getIndexesMap().entrySet()) {
-          indexTableInfos[index] =
-              new IndexTableInfo(getDatabaseName(), entry.getKey(), entry.getValue());
-          index++;
+      if (null != indexProvider) {
+        if (null != indexMetadata.getIndexesMap().get(indexProvider)) {
+          IndexTableInfo[] indexTableInfos =
+              new IndexTableInfo[indexMetadata.getIndexesMap().get(indexProvider).entrySet()
+                  .size()];
+          int index = 0;
+          if (!isIndexTable()) {
 
 Review comment:
   why this check is required ? `if (!isIndexTable()) {`
   add some comments

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406576221
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -1202,21 +1192,64 @@ public boolean isIndexTable() throws IOException {
     }
   }
 
+  public List<String> getIndexTableNames(String indexProvider) throws IOException {
+    deserializeIndexMetadata();
+    if (null != indexMetadata) {
+      return indexMetadata.getIndexTables(indexProvider);
+    } else {
+      return new ArrayList<>();
+    }
+  }
+
   public String getIndexInfo() throws IOException {
+    return getIndexInfo(null);
+  }
+
+  public IndexMetadata getIndexMetadata() throws IOException {
+    deserializeIndexMetadata();
+    return indexMetadata;
+  }
+
+  public String getIndexInfo(String indexProvider) throws IOException {
     deserializeIndexMetadata();
     if (null != indexMetadata) {
-      IndexTableInfo[] indexTableInfos =
-          new IndexTableInfo[indexMetadata.getIndexesMap().entrySet().size()];
-      int index = 0;
-      if (!isIndexTable()) {
-        for (Map.Entry<String, List<String>> entry : indexMetadata.getIndexesMap().entrySet()) {
-          indexTableInfos[index] =
-              new IndexTableInfo(getDatabaseName(), entry.getKey(), entry.getValue());
-          index++;
+      if (null != indexProvider) {
+        if (null != indexMetadata.getIndexesMap().get(indexProvider)) {
+          IndexTableInfo[] indexTableInfos =
+              new IndexTableInfo[indexMetadata.getIndexesMap().get(indexProvider).entrySet()
+                  .size()];
+          int index = 0;
+          if (!isIndexTable()) {
 
 Review comment:
   added

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405998756
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonRefreshIndexCommand.scala
 ##########
 @@ -71,30 +80,81 @@ case class CarbonRefreshIndexCommand(
   private def refreshIndex(
       sparkSession: SparkSession,
       parentTable: CarbonTable,
-      indexOp: Optional[DataMapSchema]): Unit = {
-    val schema = indexOp.get
+      indexMetaData: IndexMetadata): Unit = {
+    var indexInfo: util.Map[String, String] = new util.HashMap[String, String]()
+    val allIndexesIterator = indexMetaData.getIndexesMap.entrySet().iterator()
+    breakable {
+      while (allIndexesIterator.hasNext) {
+        val currentIndex = allIndexesIterator.next()
+        if (!currentIndex.getKey.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName)) {
 
 Review comment:
   If we are excluding SI everywhere, It is better to have separate map for each provider in `IndexMetadata`. access only Non SI maps.
   
   or you can directly lookup non SI providers

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607025357
 
 
   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] Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406576125
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonRefreshIndexCommand.scala
 ##########
 @@ -71,30 +80,81 @@ case class CarbonRefreshIndexCommand(
   private def refreshIndex(
       sparkSession: SparkSession,
       parentTable: CarbonTable,
-      indexOp: Optional[DataMapSchema]): Unit = {
-    val schema = indexOp.get
+      indexMetaData: IndexMetadata): Unit = {
+    var indexInfo: util.Map[String, String] = new util.HashMap[String, String]()
+    val allIndexesIterator = indexMetaData.getIndexesMap.entrySet().iterator()
+    breakable {
+      while (allIndexesIterator.hasNext) {
+        val currentIndex = allIndexesIterator.next()
+        if (!currentIndex.getKey.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName)) {
 
 Review comment:
   handled

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


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406575861
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/index/CarbonIndexUtil.scala
 ##########
 @@ -105,7 +105,8 @@ object CarbonIndexUtil {
     val indexMeta = carbonTable.getTableInfo.getFactTable.getTableProperties
       .get(carbonTable.getCarbonTableIdentifier.getTableId)
     val indexesTables = if (null != indexMeta) {
 
 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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612131142
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2710/
   

----------------------------------------------------------------
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] niuge01 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r408665324
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/index/CarbonIndexProvider.java
 ##########
 @@ -72,91 +72,95 @@
  * @since 1.4.0
  */
 @InterfaceAudience.Internal
-public abstract class DataMapProvider {
+public abstract class CarbonIndexProvider {
 
 Review comment:
   ```suggestion
   public abstract class IndexProvider {
   ```

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406576162
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonCreateIndexCommand.scala
 ##########
 @@ -67,128 +82,195 @@ case class CarbonCreateIndexCommand(
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
 
-    if (DataMapStoreManager.getInstance().isDataMapExist(parentTable.getTableId, indexName)) {
-      if (!ifNotExistsSet) {
-        throw new MalformedIndexCommandException(
-          s"Index with name ${ indexName } on table " +
-            s"${parentTable.getDatabaseName}.${parentTable.getTableName} already exists")
-      } else {
-        return Seq.empty
-      }
+    if (parentTable.isMVTable || parentTable.isIndexTable) {
+      throw new MalformedIndexCommandException(
+        "Cannot create index on child table `" + indexName + "`")
     }
 
     if (CarbonUtil.getFormatVersion(parentTable) != ColumnarFormatVersion.V3) {
-      throw new MalformedCarbonCommandException(s"Unsupported operation on table with " +
-                                                s"V1 or V2 format data")
+      throw new MalformedCarbonCommandException(
+        s"Unsupported operation on table with V1 or V2 format data")
     }
 
-    dataMapSchema = new DataMapSchema(indexName, indexProviderName)
+    // get metadata lock to avoid concurrent create index operations
+    val metadataLock = CarbonLockFactory.getCarbonLockObj(
+      parentTable.getAbsoluteTableIdentifier,
+      LockUsage.METADATA_LOCK)
 
-    val property = properties.map(x => (x._1.trim, x._2.trim)).asJava
-    val javaMap = new java.util.HashMap[String, String](property)
-    javaMap.put(DataMapProperty.DEFERRED_REBUILD, deferredRebuild.toString)
-    javaMap.put(CarbonCommonConstants.INDEX_COLUMNS, indexModel.columnNames.mkString(","))
-    dataMapSchema.setProperties(javaMap)
+    try {
+      if (metadataLock.lockWithRetries()) {
+        LOGGER.info(s"Acquired the metadata lock for table $dbName.$parentTableName")
+        // get carbon table again to reflect any changes during lock acquire.
+        parentTable =
+          CarbonEnv.getInstance(sparkSession).carbonMetaStore
+            .lookupRelation(Some(dbName), parentTableName)(sparkSession)
+            .asInstanceOf[CarbonRelation].carbonTable
+        if (parentTable == null) {
+          throw new MalformedIndexCommandException(errMsg)
+        }
+        val oldIndexMetaData = parentTable.getIndexMetadata
+        // check whether the column has index created already
+        if (null != oldIndexMetaData) {
+          val indexExistsInCarbon = oldIndexMetaData.getIndexTables.asScala.contains(indexName)
+          if (indexExistsInCarbon) {
+            throw new MalformedIndexCommandException(
+              "Index with name `" + indexName + "` already exists on table `" + parentTableName +
+              "`")
+          }
+        }
+        // set properties
+        indexSchema.setProperties(indexProperties)
+        provider = new IndexProvider(parentTable, indexSchema, sparkSession)
 
-    if (dataMapSchema.isIndex && parentTable == null) {
-      throw new MalformedIndexCommandException(
-        "To create index, main table is required. Use `CREATE INDEX ... ON TABLE ...` ")
-    }
-    provider = new IndexProvider(parentTable, dataMapSchema, sparkSession)
-    if (deferredRebuild && !provider.supportRebuild()) {
-      throw new MalformedIndexCommandException(
-        s"DEFERRED REFRESH is not supported on this index $indexName" +
-        s" with provider ${dataMapSchema.getProviderName}")
-    }
+        if (deferredRebuild && !provider.supportRebuild()) {
+          throw new MalformedIndexCommandException(
+          "DEFERRED REFRESH is not supported on this index " + indexModel.indexName +
+          " with provider " + indexProviderName)
+        } else if (deferredRebuild && provider.supportRebuild()) {
+          indexProperties.put(CarbonCommonConstants.INDEX_STATUS, IndexStatus.DISABLED.name())
+        }
 
-    if (parentTable.isMVTable) {
-      throw new MalformedIndexCommandException(
-        "Cannot create index on MV table " + parentTable.getTableUniqueName)
-    }
+        val isBloomFilter = CarbonIndexProvider.BLOOMFILTER.getIndexProviderName
 
 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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-610808714
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2677/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406002402
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/ShowIndexesCommand.scala
 ##########
 @@ -54,69 +62,87 @@ case class ShowIndexesCommand(
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     val carbonTable = CarbonEnv.getCarbonTable(dbNameOp, tableName)(sparkSession)
     setAuditTable(carbonTable)
-    getFileIndexInfo(carbonTable) ++ getSIInfo(sparkSession, carbonTable)
-  }
-
-  // get info for 'index datamap'
-  private def getFileIndexInfo(carbonTable: CarbonTable): Seq[Row] = {
-    val indexes = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable).asScala
-    if (indexes != null && indexes.nonEmpty) {
-      indexes.map { index =>
-        Row(
-          index.getDataMapName,
-          index.getProviderName,
-          index.getIndexColumns.mkString(","),
-          index.getPropertiesAsString,
-          index.getStatus.name(),
-          index.getSyncStatus
-        )
-      }
-    } else {
-      Seq.empty
-    }
+    getIndexInfo(sparkSession, carbonTable)
   }
 
-  // get info for SI
-  private def getSIInfo(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
+  private def getIndexInfo(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
     CarbonInternalMetastore.refreshIndexInfo(
       carbonTable.getDatabaseName, tableName, carbonTable)(sparkSession)
-    val indexesMap = CarbonInternalScalaUtil.getIndexesMap(carbonTable)
-    if (null == indexesMap) {
-      throw new Exception("Secondary index information is not loaded in main table")
-    }
-    val indexTableMap = indexesMap.asScala
-    if (indexTableMap.nonEmpty) {
-      val indexList = indexTableMap.map { indexInfo =>
-        try {
-          val isSITableEnabled = sparkSession.sessionState.catalog
-            .getTableMetadata(TableIdentifier(indexInfo._1, dbNameOp)).storage.properties
-            .getOrElse("isSITableEnabled", "true").equalsIgnoreCase("true")
-          if (isSITableEnabled) {
-            (indexInfo._1, indexInfo._2.asScala.mkString(","), "enabled")
-          } else {
-            (indexInfo._1, indexInfo._2.asScala.mkString(","), "disabled")
+    val indexesMap = CarbonIndexUtil.getIndexesMap(carbonTable)
+    if (null != indexesMap) {
+      val indexTableMap = indexesMap.asScala
+      if (indexTableMap.nonEmpty) {
+        val secondaryIndex = indexTableMap.get(CarbonIndexProvider.SI.getIndexProviderName)
+        var finalIndexList: Seq[(String, String, String, String, String, String)] = Seq.empty
+
+        if (secondaryIndex.isDefined && null != secondaryIndex.get) {
+          val siIterator = secondaryIndex.get.entrySet().iterator()
+          while (siIterator.hasNext) {
+            val indexInfo = siIterator.next()
+            try {
+              val isSITableEnabled = sparkSession.sessionState.catalog
+                .getTableMetadata(TableIdentifier(indexInfo.getKey, dbNameOp)).storage.properties
+                .getOrElse("isSITableEnabled", "true").equalsIgnoreCase("true")
+              if (isSITableEnabled) {
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants.INDEX_COLUMNS), "NA", "enabled", "NA")
+              } else {
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants
+                                     .INDEX_COLUMNS), "NA", "disabled", "NA")
+              }
+            } catch {
+              case ex: Exception =>
+                LOGGER.error(s"Access storage properties from hive failed for index table: ${
+                  indexInfo.getKey
+                }")
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants.INDEX_COLUMNS), "NA", "UNKNOWN", "NA")
+            }
+          }
+        }
+
+        indexesMap.asScala
+          .filter(map => !map._1.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName))
 
 Review comment:
   As mentioned above instead of 1 map, if we keep 3 map (each for it's index provider). No need filter logic

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-608024959
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/915/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406056076
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/index/CarbonIndexUtil.scala
 ##########
 @@ -105,7 +105,8 @@ object CarbonIndexUtil {
     val indexMeta = carbonTable.getTableInfo.getFactTable.getTableProperties
       .get(carbonTable.getCarbonTableIdentifier.getTableId)
     val indexesTables = if (null != indexMeta) {
 
 Review comment:
   ```suggestion
       val SecondaryIndexTables = if (null != indexMeta) {
   ```

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-608574551
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/923/
   

----------------------------------------------------------------
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] ajantha-bhat commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612787201
 
 
   please rebase.
   
   LGTM. 
   
   I feel two people review is better for this PR as changes are more files and chances of missing some issues are more.
   
   @jackylk , @QiangCai . @akashrn5 , @kunal642 : can anyone else review ?

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-615099872
 
 
   @ajantha-bhat it has old mv dependency. it will be removed in [PR-3692](https://github.com/apache/carbondata/pull/3692)

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611880823
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2701/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607777601
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2620/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-606700826
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/889/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406010654
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala
 ##########
 @@ -136,14 +137,16 @@ object CarbonInternalMetastore {
 
   def refreshIndexInfo(dbName: String, tableName: String,
       carbonTable: CarbonTable, needLock: Boolean = true)(sparkSession: SparkSession): Unit = {
-    val indexTableExists = CarbonInternalScalaUtil.isIndexTableExists(carbonTable)
+    val indexTableExists = CarbonIndexUtil.isIndexTableExists(carbonTable)
 
 Review comment:
   this is confusing can you add comment ? index is not an index table ?
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405274752
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/index/IndexStoreManager.java
 ##########
 @@ -820,8 +715,11 @@ public synchronized void clearInvalidIndex(CarbonTable carbonTable, List<String>
   }
 
   private boolean hasCGIndex(CarbonTable carbonTable) throws IOException {
-    for (TableIndex tableIndex : getAllVisibleIndexes(carbonTable)) {
-      if (tableIndex.getIndexFactory().getDataMapLevel().equals(IndexLevel.CG)) {
+    if (null == carbonTable) {
 
 Review comment:
   Table cannot be null here

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611501936
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/979/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-606553028
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2596/
   

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406575910
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/SILoadEventListenerForFailedSegments.scala
 ##########
 @@ -67,10 +67,11 @@ class SILoadEventListenerForFailedSegments extends OperationEventListener with L
           .deserialize(carbonTable.getTableInfo.getFactTable.getTableProperties
             .get(carbonTable.getCarbonTableIdentifier.getTableId))
         val mainTableDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+        val providerName = CarbonIndexProvider.SI.getIndexProviderName
 
 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] Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r408695450
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/index/CarbonIndexProvider.java
 ##########
 @@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.metadata.index;
+
+public enum CarbonIndexProvider {
 
 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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611880284
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/988/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611469098
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/978/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-610813477
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/965/
   

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406576178
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonCreateIndexCommand.scala
 ##########
 @@ -67,128 +82,195 @@ case class CarbonCreateIndexCommand(
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
 
-    if (DataMapStoreManager.getInstance().isDataMapExist(parentTable.getTableId, indexName)) {
-      if (!ifNotExistsSet) {
-        throw new MalformedIndexCommandException(
-          s"Index with name ${ indexName } on table " +
-            s"${parentTable.getDatabaseName}.${parentTable.getTableName} already exists")
-      } else {
-        return Seq.empty
-      }
+    if (parentTable.isMVTable || parentTable.isIndexTable) {
+      throw new MalformedIndexCommandException(
+        "Cannot create index on child table `" + indexName + "`")
     }
 
     if (CarbonUtil.getFormatVersion(parentTable) != ColumnarFormatVersion.V3) {
-      throw new MalformedCarbonCommandException(s"Unsupported operation on table with " +
-                                                s"V1 or V2 format data")
+      throw new MalformedCarbonCommandException(
+        s"Unsupported operation on table with V1 or V2 format data")
     }
 
-    dataMapSchema = new DataMapSchema(indexName, indexProviderName)
+    // get metadata lock to avoid concurrent create index operations
+    val metadataLock = CarbonLockFactory.getCarbonLockObj(
+      parentTable.getAbsoluteTableIdentifier,
+      LockUsage.METADATA_LOCK)
 
-    val property = properties.map(x => (x._1.trim, x._2.trim)).asJava
-    val javaMap = new java.util.HashMap[String, String](property)
-    javaMap.put(DataMapProperty.DEFERRED_REBUILD, deferredRebuild.toString)
-    javaMap.put(CarbonCommonConstants.INDEX_COLUMNS, indexModel.columnNames.mkString(","))
-    dataMapSchema.setProperties(javaMap)
+    try {
+      if (metadataLock.lockWithRetries()) {
+        LOGGER.info(s"Acquired the metadata lock for table $dbName.$parentTableName")
+        // get carbon table again to reflect any changes during lock acquire.
+        parentTable =
+          CarbonEnv.getInstance(sparkSession).carbonMetaStore
+            .lookupRelation(Some(dbName), parentTableName)(sparkSession)
+            .asInstanceOf[CarbonRelation].carbonTable
+        if (parentTable == null) {
+          throw new MalformedIndexCommandException(errMsg)
+        }
+        val oldIndexMetaData = parentTable.getIndexMetadata
+        // check whether the column has index created already
+        if (null != oldIndexMetaData) {
+          val indexExistsInCarbon = oldIndexMetaData.getIndexTables.asScala.contains(indexName)
+          if (indexExistsInCarbon) {
+            throw new MalformedIndexCommandException(
+              "Index with name `" + indexName + "` already exists on table `" + parentTableName +
+              "`")
+          }
+        }
+        // set properties
+        indexSchema.setProperties(indexProperties)
+        provider = new IndexProvider(parentTable, indexSchema, sparkSession)
 
-    if (dataMapSchema.isIndex && parentTable == null) {
-      throw new MalformedIndexCommandException(
-        "To create index, main table is required. Use `CREATE INDEX ... ON TABLE ...` ")
-    }
-    provider = new IndexProvider(parentTable, dataMapSchema, sparkSession)
-    if (deferredRebuild && !provider.supportRebuild()) {
-      throw new MalformedIndexCommandException(
-        s"DEFERRED REFRESH is not supported on this index $indexName" +
-        s" with provider ${dataMapSchema.getProviderName}")
-    }
+        if (deferredRebuild && !provider.supportRebuild()) {
+          throw new MalformedIndexCommandException(
+          "DEFERRED REFRESH is not supported on this index " + indexModel.indexName +
+          " with provider " + indexProviderName)
+        } else if (deferredRebuild && provider.supportRebuild()) {
+          indexProperties.put(CarbonCommonConstants.INDEX_STATUS, IndexStatus.DISABLED.name())
+        }
 
-    if (parentTable.isMVTable) {
-      throw new MalformedIndexCommandException(
-        "Cannot create index on MV table " + parentTable.getTableUniqueName)
-    }
+        val isBloomFilter = CarbonIndexProvider.BLOOMFILTER.getIndexProviderName
+          .equalsIgnoreCase(indexProviderName)
 
-    if (parentTable.isIndexTable) {
-      throw new MalformedIndexCommandException(
-        "Cannot create index on Secondary Index table")
-    }
+        val existingIndexColumn4ThisProvider =
 
 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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612117361
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/998/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612795589
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1007/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405273233
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -1536,12 +1536,12 @@ private CarbonCommonConstants() {
   //////////////////////////////////////////////////////////////////////////////////////////
 
   /**
-   * key prefix for set command. 'carbon.datamap.visible.dbName.tableName.dmName = false' means
+   * key prefix for set command. 'carbon.index.visible.dbName.tableName.dmName = false' means
    * that the query on 'dbName.table' will not use the datamap 'dmName'
 
 Review comment:
   should have not combined refactoring and functionality PR together. It is hard to review 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] Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406575949
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala
 ##########
 @@ -192,6 +226,42 @@ object CarbonInternalMetastore {
           LOGGER.error(e.getMessage)
       }
     }
+    if (null != indexExists) {
+      if (null != carbonTable && (null == indexExists || indexExists.toBoolean)) {
 
 Review comment:
   Handled

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


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-613610631
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2743/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611937488
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2705/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406054448
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/SILoadEventListenerForFailedSegments.scala
 ##########
 @@ -67,10 +67,11 @@ class SILoadEventListenerForFailedSegments extends OperationEventListener with L
           .deserialize(carbonTable.getTableInfo.getFactTable.getTableProperties
             .get(carbonTable.getCarbonTableIdentifier.getTableId))
         val mainTableDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+        val providerName = CarbonIndexProvider.SI.getIndexProviderName
 
 Review comment:
   ```suggestion
           val secondaryIndexProvider = CarbonIndexProvider.SI.getIndexProviderName
   ```

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406010654
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala
 ##########
 @@ -136,14 +137,16 @@ object CarbonInternalMetastore {
 
   def refreshIndexInfo(dbName: String, tableName: String,
       carbonTable: CarbonTable, needLock: Boolean = true)(sparkSession: SparkSession): Unit = {
-    val indexTableExists = CarbonInternalScalaUtil.isIndexTableExists(carbonTable)
+    val indexTableExists = CarbonIndexUtil.isIndexTableExists(carbonTable)
 
 Review comment:
   this is confusing can you add comment ? index is not an index table ?

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405989456
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/datamap/IndexRebuildRDD.scala
 ##########
 @@ -101,9 +103,11 @@ object IndexRebuildRDD {
       sparkSession,
       new RefreshResultImpl(),
       carbonTable.getTableInfo,
+      schema,
       schema.getDataMapName,
       indexedCarbonColumns.asScala.toArray,
-      segments2DmStorePath.keySet
+      segments2DmStorePath.keySet,
+      schema.getProviderName
 
 Review comment:
   just passing schema is enough, 
   schema.getProviderName, schema.getDataMapName, can be obtained inside from this

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405995246
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonCreateIndexCommand.scala
 ##########
 @@ -67,128 +82,195 @@ case class CarbonCreateIndexCommand(
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
 
-    if (DataMapStoreManager.getInstance().isDataMapExist(parentTable.getTableId, indexName)) {
-      if (!ifNotExistsSet) {
-        throw new MalformedIndexCommandException(
-          s"Index with name ${ indexName } on table " +
-            s"${parentTable.getDatabaseName}.${parentTable.getTableName} already exists")
-      } else {
-        return Seq.empty
-      }
+    if (parentTable.isMVTable || parentTable.isIndexTable) {
+      throw new MalformedIndexCommandException(
+        "Cannot create index on child table `" + indexName + "`")
     }
 
     if (CarbonUtil.getFormatVersion(parentTable) != ColumnarFormatVersion.V3) {
-      throw new MalformedCarbonCommandException(s"Unsupported operation on table with " +
-                                                s"V1 or V2 format data")
+      throw new MalformedCarbonCommandException(
+        s"Unsupported operation on table with V1 or V2 format data")
     }
 
-    dataMapSchema = new DataMapSchema(indexName, indexProviderName)
+    // get metadata lock to avoid concurrent create index operations
+    val metadataLock = CarbonLockFactory.getCarbonLockObj(
+      parentTable.getAbsoluteTableIdentifier,
+      LockUsage.METADATA_LOCK)
 
-    val property = properties.map(x => (x._1.trim, x._2.trim)).asJava
-    val javaMap = new java.util.HashMap[String, String](property)
-    javaMap.put(DataMapProperty.DEFERRED_REBUILD, deferredRebuild.toString)
-    javaMap.put(CarbonCommonConstants.INDEX_COLUMNS, indexModel.columnNames.mkString(","))
-    dataMapSchema.setProperties(javaMap)
+    try {
+      if (metadataLock.lockWithRetries()) {
+        LOGGER.info(s"Acquired the metadata lock for table $dbName.$parentTableName")
+        // get carbon table again to reflect any changes during lock acquire.
+        parentTable =
+          CarbonEnv.getInstance(sparkSession).carbonMetaStore
+            .lookupRelation(Some(dbName), parentTableName)(sparkSession)
+            .asInstanceOf[CarbonRelation].carbonTable
+        if (parentTable == null) {
+          throw new MalformedIndexCommandException(errMsg)
+        }
+        val oldIndexMetaData = parentTable.getIndexMetadata
+        // check whether the column has index created already
+        if (null != oldIndexMetaData) {
+          val indexExistsInCarbon = oldIndexMetaData.getIndexTables.asScala.contains(indexName)
+          if (indexExistsInCarbon) {
+            throw new MalformedIndexCommandException(
+              "Index with name `" + indexName + "` already exists on table `" + parentTableName +
+              "`")
+          }
+        }
+        // set properties
+        indexSchema.setProperties(indexProperties)
+        provider = new IndexProvider(parentTable, indexSchema, sparkSession)
 
-    if (dataMapSchema.isIndex && parentTable == null) {
-      throw new MalformedIndexCommandException(
-        "To create index, main table is required. Use `CREATE INDEX ... ON TABLE ...` ")
-    }
-    provider = new IndexProvider(parentTable, dataMapSchema, sparkSession)
-    if (deferredRebuild && !provider.supportRebuild()) {
-      throw new MalformedIndexCommandException(
-        s"DEFERRED REFRESH is not supported on this index $indexName" +
-        s" with provider ${dataMapSchema.getProviderName}")
-    }
+        if (deferredRebuild && !provider.supportRebuild()) {
+          throw new MalformedIndexCommandException(
+          "DEFERRED REFRESH is not supported on this index " + indexModel.indexName +
+          " with provider " + indexProviderName)
+        } else if (deferredRebuild && provider.supportRebuild()) {
+          indexProperties.put(CarbonCommonConstants.INDEX_STATUS, IndexStatus.DISABLED.name())
+        }
 
-    if (parentTable.isMVTable) {
-      throw new MalformedIndexCommandException(
-        "Cannot create index on MV table " + parentTable.getTableUniqueName)
-    }
+        val isBloomFilter = CarbonIndexProvider.BLOOMFILTER.getIndexProviderName
 
 Review comment:
   keep definition and usage together. move to line 152

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-615183853
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1052/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-608550918
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2632/
   

----------------------------------------------------------------
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] ajantha-bhat commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-610773012
 
 
   @Indhumathi27 : Just checked few files on high level. Couldn't focus on functionality as there is attracting 300 file rename changes. Suggest to separate rename and functionality(multi-tenant) PR

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-615191103
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2765/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-608429691
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2628/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607353473
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/902/
   

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406783323
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -505,7 +505,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   /**
-   * INSERT INTO [dbName.]tableName STAGE [OPTIONS (key1=value1, key2=value2, ...)]
+   * INSERT INTO [dbName.]indexName STAGE [OPTIONS (key1=value1, key2=value2, ...)]
 
 Review comment:
   Reverted

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612794400
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2719/
   

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406576435
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexFactory.java
 ##########
 @@ -187,7 +187,8 @@ public void deleteSegmentIndexData(String segmentNo) throws IOException {
    * 4. INDEX_COLUMNS should be exists in table columns
    */
   public void validate() throws MalformedIndexCommandException {
-    List<CarbonColumn> indexColumns = carbonTable.getIndexedColumns(dataMapSchema);
+    List<CarbonColumn> indexColumns =
+        carbonTable.getIndexedColumns(dataMapSchema.getIndexColumns());
 
 Review comment:
   handled

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


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-613608355
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1030/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607846285
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/912/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-606108217
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2594/
   

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405294912
 
 

 ##########
 File path: format/src/main/thrift/schema.thrift
 ##########
 @@ -203,9 +203,9 @@ struct ParentColumnTableRelation {
    3: required string columnName
 }
 
-struct DataMapSchema  {
+struct IndexSchema  {
 
 Review comment:
   These changes added in `schema.thrift` are to support pre-aggregate. I will remove those changes from `schema.thrift` file

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611621773
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/983/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611942208
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/992/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-613523209
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1028/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-608621910
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2636/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-610775682
 
 
   @ajantha-bhat there are seperate commits for support muti-tenant and datamap refactor in this PR. If still difficult to review, i can raise one more PR

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-608028688
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2624/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406054448
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/SILoadEventListenerForFailedSegments.scala
 ##########
 @@ -67,10 +67,11 @@ class SILoadEventListenerForFailedSegments extends OperationEventListener with L
           .deserialize(carbonTable.getTableInfo.getFactTable.getTableProperties
             .get(carbonTable.getCarbonTableIdentifier.getTableId))
         val mainTableDetails = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+        val providerName = CarbonIndexProvider.SI.getIndexProviderName
 
 Review comment:
   ```suggestion
           val secondaryIndexProviderName = CarbonIndexProvider.SI.getIndexProviderName
   ```

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406576037
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonRefreshIndexCommand.scala
 ##########
 @@ -71,30 +80,81 @@ case class CarbonRefreshIndexCommand(
   private def refreshIndex(
       sparkSession: SparkSession,
       parentTable: CarbonTable,
-      indexOp: Optional[DataMapSchema]): Unit = {
-    val schema = indexOp.get
+      indexMetaData: IndexMetadata): Unit = {
+    var indexInfo: util.Map[String, String] = new util.HashMap[String, String]()
+    val allIndexesIterator = indexMetaData.getIndexesMap.entrySet().iterator()
+    breakable {
+      while (allIndexesIterator.hasNext) {
+        val currentIndex = allIndexesIterator.next()
+        if (!currentIndex.getKey.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName)) {
+          val indexIterator = currentIndex.getValue.entrySet().iterator()
+          while (indexIterator.hasNext) {
+            val indexEntry = indexIterator.next()
+            if (indexEntry.getKey.equalsIgnoreCase(indexName)) {
+              indexInfo = indexEntry.getValue
+              break()
+            }
+          }
+        }
+      }
+    }
+    if (indexInfo.isEmpty) {
+      throw new MalformedIndexCommandException(
+        "Index with name `" + indexName + "` is not present" +
+        "on table `" + parentTable.getTableName + "`")
+    }
+    val indexProviderName = indexInfo.get(CarbonCommonConstants.INDEX_PROVIDER)
+    val schema = new DataMapSchema(indexName, indexProviderName)
+    schema.setProperties(indexInfo)
     if (!schema.isLazy) {
       throw new MalformedIndexCommandException(
         s"Non-lazy index $indexName does not support manual refresh")
     }
 
     val provider = DataMapManager.get().getDataMapProvider(parentTable, schema, sparkSession)
     provider.rebuild()
+    // enable bloom or lucene index
+    // get metadata lock to avoid concurrent create index operations
+    val metadataLock = CarbonLockFactory.getCarbonLockObj(
+      parentTable.getAbsoluteTableIdentifier,
+      LockUsage.METADATA_LOCK)
+    try {
+      if (metadataLock.lockWithRetries()) {
+        LOGGER.info(s"Acquired the metadata lock for table " +
+                    s"${ parentTable.getDatabaseName}.${ parentTable.getTableName }")
+        val oldIndexInfo = parentTable.getIndexInfo
+        val updatedIndexInfo = IndexTableInfo.enableIndex(oldIndexInfo, indexName)
+
+        // set index information in parent table
+        val parentIndexMetadata =
+          IndexMetadata.deserialize(parentTable.getTableInfo.getFactTable.getTableProperties
 
 Review comment:
   Handled

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


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405994595
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonCreateIndexCommand.scala
 ##########
 @@ -67,128 +82,195 @@ case class CarbonCreateIndexCommand(
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
 
-    if (DataMapStoreManager.getInstance().isDataMapExist(parentTable.getTableId, indexName)) {
-      if (!ifNotExistsSet) {
-        throw new MalformedIndexCommandException(
-          s"Index with name ${ indexName } on table " +
-            s"${parentTable.getDatabaseName}.${parentTable.getTableName} already exists")
-      } else {
-        return Seq.empty
-      }
+    if (parentTable.isMVTable || parentTable.isIndexTable) {
+      throw new MalformedIndexCommandException(
+        "Cannot create index on child table `" + indexName + "`")
     }
 
     if (CarbonUtil.getFormatVersion(parentTable) != ColumnarFormatVersion.V3) {
-      throw new MalformedCarbonCommandException(s"Unsupported operation on table with " +
-                                                s"V1 or V2 format data")
+      throw new MalformedCarbonCommandException(
+        s"Unsupported operation on table with V1 or V2 format data")
     }
 
-    dataMapSchema = new DataMapSchema(indexName, indexProviderName)
+    // get metadata lock to avoid concurrent create index operations
+    val metadataLock = CarbonLockFactory.getCarbonLockObj(
+      parentTable.getAbsoluteTableIdentifier,
+      LockUsage.METADATA_LOCK)
 
-    val property = properties.map(x => (x._1.trim, x._2.trim)).asJava
-    val javaMap = new java.util.HashMap[String, String](property)
-    javaMap.put(DataMapProperty.DEFERRED_REBUILD, deferredRebuild.toString)
-    javaMap.put(CarbonCommonConstants.INDEX_COLUMNS, indexModel.columnNames.mkString(","))
-    dataMapSchema.setProperties(javaMap)
+    try {
+      if (metadataLock.lockWithRetries()) {
+        LOGGER.info(s"Acquired the metadata lock for table $dbName.$parentTableName")
+        // get carbon table again to reflect any changes during lock acquire.
+        parentTable =
+          CarbonEnv.getInstance(sparkSession).carbonMetaStore
+            .lookupRelation(Some(dbName), parentTableName)(sparkSession)
+            .asInstanceOf[CarbonRelation].carbonTable
+        if (parentTable == null) {
+          throw new MalformedIndexCommandException(errMsg)
+        }
+        val oldIndexMetaData = parentTable.getIndexMetadata
+        // check whether the column has index created already
+        if (null != oldIndexMetaData) {
+          val indexExistsInCarbon = oldIndexMetaData.getIndexTables.asScala.contains(indexName)
+          if (indexExistsInCarbon) {
+            throw new MalformedIndexCommandException(
+              "Index with name `" + indexName + "` already exists on table `" + parentTableName +
+              "`")
+          }
+        }
+        // set properties
+        indexSchema.setProperties(indexProperties)
+        provider = new IndexProvider(parentTable, indexSchema, sparkSession)
 
-    if (dataMapSchema.isIndex && parentTable == null) {
-      throw new MalformedIndexCommandException(
-        "To create index, main table is required. Use `CREATE INDEX ... ON TABLE ...` ")
-    }
-    provider = new IndexProvider(parentTable, dataMapSchema, sparkSession)
-    if (deferredRebuild && !provider.supportRebuild()) {
-      throw new MalformedIndexCommandException(
-        s"DEFERRED REFRESH is not supported on this index $indexName" +
-        s" with provider ${dataMapSchema.getProviderName}")
-    }
+        if (deferredRebuild && !provider.supportRebuild()) {
+          throw new MalformedIndexCommandException(
+          "DEFERRED REFRESH is not supported on this index " + indexModel.indexName +
+          " with provider " + indexProviderName)
+        } else if (deferredRebuild && provider.supportRebuild()) {
+          indexProperties.put(CarbonCommonConstants.INDEX_STATUS, IndexStatus.DISABLED.name())
+        }
 
-    if (parentTable.isMVTable) {
-      throw new MalformedIndexCommandException(
-        "Cannot create index on MV table " + parentTable.getTableUniqueName)
-    }
+        val isBloomFilter = CarbonIndexProvider.BLOOMFILTER.getIndexProviderName
+          .equalsIgnoreCase(indexProviderName)
 
-    if (parentTable.isIndexTable) {
-      throw new MalformedIndexCommandException(
-        "Cannot create index on Secondary Index table")
-    }
+        val existingIndexColumn4ThisProvider =
 
 Review comment:
   *for

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406066860
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/ShowIndexesCommand.scala
 ##########
 @@ -54,69 +62,87 @@ case class ShowIndexesCommand(
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     val carbonTable = CarbonEnv.getCarbonTable(dbNameOp, tableName)(sparkSession)
     setAuditTable(carbonTable)
-    getFileIndexInfo(carbonTable) ++ getSIInfo(sparkSession, carbonTable)
-  }
-
-  // get info for 'index datamap'
-  private def getFileIndexInfo(carbonTable: CarbonTable): Seq[Row] = {
-    val indexes = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable).asScala
-    if (indexes != null && indexes.nonEmpty) {
-      indexes.map { index =>
-        Row(
-          index.getDataMapName,
-          index.getProviderName,
-          index.getIndexColumns.mkString(","),
-          index.getPropertiesAsString,
-          index.getStatus.name(),
-          index.getSyncStatus
-        )
-      }
-    } else {
-      Seq.empty
-    }
+    getIndexInfo(sparkSession, carbonTable)
   }
 
-  // get info for SI
-  private def getSIInfo(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
+  private def getIndexInfo(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
     CarbonInternalMetastore.refreshIndexInfo(
       carbonTable.getDatabaseName, tableName, carbonTable)(sparkSession)
-    val indexesMap = CarbonInternalScalaUtil.getIndexesMap(carbonTable)
-    if (null == indexesMap) {
-      throw new Exception("Secondary index information is not loaded in main table")
-    }
-    val indexTableMap = indexesMap.asScala
-    if (indexTableMap.nonEmpty) {
-      val indexList = indexTableMap.map { indexInfo =>
-        try {
-          val isSITableEnabled = sparkSession.sessionState.catalog
-            .getTableMetadata(TableIdentifier(indexInfo._1, dbNameOp)).storage.properties
-            .getOrElse("isSITableEnabled", "true").equalsIgnoreCase("true")
-          if (isSITableEnabled) {
-            (indexInfo._1, indexInfo._2.asScala.mkString(","), "enabled")
-          } else {
-            (indexInfo._1, indexInfo._2.asScala.mkString(","), "disabled")
+    val indexesMap = CarbonIndexUtil.getIndexesMap(carbonTable)
+    if (null != indexesMap) {
+      val indexTableMap = indexesMap.asScala
+      if (indexTableMap.nonEmpty) {
+        val secondaryIndex = indexTableMap.get(CarbonIndexProvider.SI.getIndexProviderName)
+        var finalIndexList: Seq[(String, String, String, String, String, String)] = Seq.empty
+
+        if (secondaryIndex.isDefined && null != secondaryIndex.get) {
+          val siIterator = secondaryIndex.get.entrySet().iterator()
+          while (siIterator.hasNext) {
+            val indexInfo = siIterator.next()
+            try {
+              val isSITableEnabled = sparkSession.sessionState.catalog
+                .getTableMetadata(TableIdentifier(indexInfo.getKey, dbNameOp)).storage.properties
+                .getOrElse("isSITableEnabled", "true").equalsIgnoreCase("true")
+              if (isSITableEnabled) {
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants.INDEX_COLUMNS), "NA", "enabled", "NA")
+              } else {
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants
+                                     .INDEX_COLUMNS), "NA", "disabled", "NA")
+              }
+            } catch {
+              case ex: Exception =>
+                LOGGER.error(s"Access storage properties from hive failed for index table: ${
+                  indexInfo.getKey
+                }")
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants.INDEX_COLUMNS), "NA", "UNKNOWN", "NA")
+            }
+          }
+        }
+
+        indexesMap.asScala
+          .filter(map => !map._1.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName))
 
 Review comment:
   No. Now you have map1(map2(map3)))
   SI map - map2(map3)
   Cgfgmap - map2(map3)
   Instead of 3 level map it is two level map. It saves default map location size of map1.
   
   More over size, no need of lookup and code may be cleaner. You can check. 

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611495688
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2692/
   

----------------------------------------------------------------
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] ajantha-bhat commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611423944
 
 
   We are making cg, fg index like secondary index. But SI is a table, cg fg is not a table. So need separate handling. Which makes code more complex and hard to maintain. @jackylk @qiangcai @indhumathi27

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405971778
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -101,49 +100,29 @@ private DataMapStoreManager() {
   }
 
   /**
-   * It only gives the visible datamaps
-   */
-  List<TableIndex> getAllVisibleIndexes(CarbonTable carbonTable) throws IOException {
-    CarbonSessionInfo sessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo();
-    List<TableIndex> allDataMaps = getAllIndexes(carbonTable);
-    Iterator<TableIndex> dataMapIterator = allDataMaps.iterator();
-    while (dataMapIterator.hasNext()) {
-      TableIndex dataMap = dataMapIterator.next();
-      String dbName = carbonTable.getDatabaseName();
-      String tableName = carbonTable.getTableName();
-      String dmName = dataMap.getDataMapSchema().getDataMapName();
-      // TODO: need support get the visible status of datamap without sessionInfo in the future
-      if (sessionInfo != null) {
-        boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
-            String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
-                dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
-        if (!isDmVisible) {
-          LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
-              dmName, dbName, tableName));
-          dataMapIterator.remove();
-        }
-      } else {
-        String message = "Carbon session info is null";
-        LOGGER.info(message);
-      }
-    }
-    return allDataMaps;
-  }
-
-  /**
-   * It gives all indexes except the default index.
+   * It gives all indexes except the default index and secondary index.
+   * Collect's Coarse grain and Fine grain indexes on a table
    *
    * @return
    */
   public List<TableIndex> getAllIndexes(CarbonTable carbonTable) throws IOException {
-    List<DataMapSchema> dataMapSchemas = getDataMapSchemasOfTable(carbonTable);
+    String indexMeta = carbonTable.getTableInfo().getFactTable().getTableProperties()
+        .get(carbonTable.getCarbonTableIdentifier().getTableId());
 
 Review comment:
   why use table id for key for this table property ? why not call key as "index_meta" ?

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-608625942
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/927/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405986131
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -1202,21 +1192,64 @@ public boolean isIndexTable() throws IOException {
     }
   }
 
+  public List<String> getIndexTableNames(String indexProvider) throws IOException {
+    deserializeIndexMetadata();
+    if (null != indexMetadata) {
+      return indexMetadata.getIndexTables(indexProvider);
+    } else {
+      return new ArrayList<>();
+    }
+  }
+
   public String getIndexInfo() throws IOException {
+    return getIndexInfo(null);
+  }
+
+  public IndexMetadata getIndexMetadata() throws IOException {
+    deserializeIndexMetadata();
+    return indexMetadata;
+  }
+
+  public String getIndexInfo(String indexProvider) throws IOException {
     deserializeIndexMetadata();
     if (null != indexMetadata) {
-      IndexTableInfo[] indexTableInfos =
-          new IndexTableInfo[indexMetadata.getIndexesMap().entrySet().size()];
-      int index = 0;
-      if (!isIndexTable()) {
-        for (Map.Entry<String, List<String>> entry : indexMetadata.getIndexesMap().entrySet()) {
-          indexTableInfos[index] =
-              new IndexTableInfo(getDatabaseName(), entry.getKey(), entry.getValue());
-          index++;
+      if (null != indexProvider) {
+        if (null != indexMetadata.getIndexesMap().get(indexProvider)) {
+          IndexTableInfo[] indexTableInfos =
+              new IndexTableInfo[indexMetadata.getIndexesMap().get(indexProvider).entrySet()
+                  .size()];
+          int index = 0;
+          if (!isIndexTable()) {
 
 Review comment:
   why this check is required ? `if (!isIndexTable()) {`

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607854616
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2621/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612891431
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1013/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607068826
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2600/
   

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406575887
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/SILoadEventListener.scala
 ##########
 @@ -61,21 +61,18 @@ class SILoadEventListener extends OperationEventListener with Logging {
         val indexMetadata = IndexMetadata
           .deserialize(carbonTable.getTableInfo.getFactTable.getTableProperties
             .get(carbonTable.getCarbonTableIdentifier.getTableId))
-        if (null != indexMetadata && null != indexMetadata.getIndexesMap && null != indexMetadata
-          .getIndexesMap.get(CarbonIndexProvider.SI.getIndexProviderName)) {
+        val provider = CarbonIndexProvider.SI.getIndexProviderName
 
 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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607348609
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2609/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-610887402
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2681/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406055127
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/SILoadEventListener.scala
 ##########
 @@ -61,21 +61,18 @@ class SILoadEventListener extends OperationEventListener with Logging {
         val indexMetadata = IndexMetadata
           .deserialize(carbonTable.getTableInfo.getFactTable.getTableProperties
             .get(carbonTable.getCarbonTableIdentifier.getTableId))
-        if (null != indexMetadata && null != indexMetadata.getIndexesMap && null != indexMetadata
-          .getIndexesMap.get(CarbonIndexProvider.SI.getIndexProviderName)) {
+        val provider = CarbonIndexProvider.SI.getIndexProviderName
 
 Review comment:
   ```suggestion
           val secondaryIndexProvider = CarbonIndexProvider.SI.getIndexProviderName
   ```

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405286075
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/index/IndexChooser.java
 ##########
 @@ -68,15 +67,14 @@
   public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
     // read all indexes for this table and populate CG and FG index list
-    List<TableIndex> visibleIndexes =
-        DataMapStoreManager.getInstance().getAllVisibleIndexes(carbonTable);
-    Map<String, DataMapStatusDetail> map = DataMapStatusManager.readDataMapStatusMap();
+    List<TableIndex> visibleIndexes = carbonTable.getAllVisibleIndexes();
     cgIndexes = new ArrayList<>(visibleIndexes.size());
     fgIndexes = new ArrayList<>(visibleIndexes.size());
     for (TableIndex visibleIndex : visibleIndexes) {
-      DataMapStatusDetail status = map.get(visibleIndex.getDataMapSchema().getDataMapName());
-      if (status != null && status.isEnabled()) {
-        IndexLevel level = visibleIndex.getIndexFactory().getDataMapLevel();
+      if (visibleIndex.getIndexSchema().getProperties().get(CarbonCommonConstants.INDEX_STATUS)
 
 Review comment:
   DatamapSchemas from old store will be not taken/considered with current code

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405268051
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -1536,12 +1536,12 @@ private CarbonCommonConstants() {
   //////////////////////////////////////////////////////////////////////////////////////////
 
   /**
-   * key prefix for set command. 'carbon.datamap.visible.dbName.tableName.dmName = false' means
+   * key prefix for set command. 'carbon.index.visible.dbName.tableName.dmName = false' means
    * that the query on 'dbName.table' will not use the datamap 'dmName'
 
 Review comment:
   ```suggestion
      * that the query on 'dbName.table' will not use the index 'dmName'
   ```

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612820846
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2723/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405975524
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/dev/IndexFactory.java
 ##########
 @@ -187,7 +187,8 @@ public void deleteSegmentIndexData(String segmentNo) throws IOException {
    * 4. INDEX_COLUMNS should be exists in table columns
    */
   public void validate() throws MalformedIndexCommandException {
-    List<CarbonColumn> indexColumns = carbonTable.getIndexedColumns(dataMapSchema);
+    List<CarbonColumn> indexColumns =
+        carbonTable.getIndexedColumns(dataMapSchema.getIndexColumns());
 
 Review comment:
   datamap schema should have already validated columns. If columns are dropped, we can update the datamap schema. Everytime checking columns in datamap schema is valid or not from all the flows is not efficient 

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406575980
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala
 ##########
 @@ -136,14 +137,16 @@ object CarbonInternalMetastore {
 
   def refreshIndexInfo(dbName: String, tableName: String,
       carbonTable: CarbonTable, needLock: Boolean = true)(sparkSession: SparkSession): Unit = {
-    val indexTableExists = CarbonInternalScalaUtil.isIndexTableExists(carbonTable)
+    val indexTableExists = CarbonIndexUtil.isIndexTableExists(carbonTable)
 
 Review comment:
   Added

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611637209
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2696/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405279772
 
 

 ##########
 File path: format/src/main/thrift/schema.thrift
 ##########
 @@ -203,9 +203,9 @@ struct ParentColumnTableRelation {
    3: required string columnName
 }
 
-struct DataMapSchema  {
+struct IndexSchema  {
 
 Review comment:
   @niuge01 , @QiangCai : please check in previous merged PR also for compatibility issues. better to test old store once.

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405973637
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -101,49 +100,29 @@ private DataMapStoreManager() {
   }
 
   /**
-   * It only gives the visible datamaps
-   */
-  List<TableIndex> getAllVisibleIndexes(CarbonTable carbonTable) throws IOException {
-    CarbonSessionInfo sessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo();
-    List<TableIndex> allDataMaps = getAllIndexes(carbonTable);
-    Iterator<TableIndex> dataMapIterator = allDataMaps.iterator();
-    while (dataMapIterator.hasNext()) {
-      TableIndex dataMap = dataMapIterator.next();
-      String dbName = carbonTable.getDatabaseName();
-      String tableName = carbonTable.getTableName();
-      String dmName = dataMap.getDataMapSchema().getDataMapName();
-      // TODO: need support get the visible status of datamap without sessionInfo in the future
-      if (sessionInfo != null) {
-        boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
-            String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
-                dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
-        if (!isDmVisible) {
-          LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
-              dmName, dbName, tableName));
-          dataMapIterator.remove();
-        }
-      } else {
-        String message = "Carbon session info is null";
-        LOGGER.info(message);
-      }
-    }
-    return allDataMaps;
-  }
-
-  /**
-   * It gives all indexes except the default index.
+   * It gives all indexes except the default index and secondary index.
+   * Collect's Coarse grain and Fine grain indexes on a table
    *
    * @return
    */
   public List<TableIndex> getAllIndexes(CarbonTable carbonTable) throws IOException {
-    List<DataMapSchema> dataMapSchemas = getDataMapSchemasOfTable(carbonTable);
+    String indexMeta = carbonTable.getTableInfo().getFactTable().getTableProperties()
+        .get(carbonTable.getCarbonTableIdentifier().getTableId());
 
 Review comment:
   Secondary index info is already stored in table with key as table-id. Used the same

----------------------------------------------------------------
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] niuge01 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r408666681
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/index/CarbonIndexProvider.java
 ##########
 @@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.metadata.index;
+
+public enum CarbonIndexProvider {
 
 Review comment:
   ```suggestion
   public enum IndexType {
   ```

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-612891449
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2725/
   

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r408668919
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/index/CarbonIndexProvider.java
 ##########
 @@ -72,91 +72,95 @@
  * @since 1.4.0
  */
 @InterfaceAudience.Internal
-public abstract class DataMapProvider {
+public abstract class CarbonIndexProvider {
 
 Review comment:
   Already we have a class with same name. Cannot rename with same

----------------------------------------------------------------
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 issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
jackylk commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-615270025
 
 
   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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406777679
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -505,7 +505,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   /**
-   * INSERT INTO [dbName.]tableName STAGE [OPTIONS (key1=value1, key2=value2, ...)]
+   * INSERT INTO [dbName.]indexName STAGE [OPTIONS (key1=value1, key2=value2, ...)]
 
 Review comment:
   this is still a table name.

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-613278436
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2733/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-610904927
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/969/
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-607074142
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/892/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405999033
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonRefreshIndexCommand.scala
 ##########
 @@ -71,30 +80,81 @@ case class CarbonRefreshIndexCommand(
   private def refreshIndex(
       sparkSession: SparkSession,
       parentTable: CarbonTable,
-      indexOp: Optional[DataMapSchema]): Unit = {
-    val schema = indexOp.get
+      indexMetaData: IndexMetadata): Unit = {
+    var indexInfo: util.Map[String, String] = new util.HashMap[String, String]()
+    val allIndexesIterator = indexMetaData.getIndexesMap.entrySet().iterator()
+    breakable {
+      while (allIndexesIterator.hasNext) {
+        val currentIndex = allIndexesIterator.next()
+        if (!currentIndex.getKey.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName)) {
+          val indexIterator = currentIndex.getValue.entrySet().iterator()
+          while (indexIterator.hasNext) {
+            val indexEntry = indexIterator.next()
+            if (indexEntry.getKey.equalsIgnoreCase(indexName)) {
 
 Review comment:
   it's map. why are you looping every element. Just check containsKey() no need of iterator here

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405976891
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -101,49 +100,29 @@ private DataMapStoreManager() {
   }
 
   /**
-   * It only gives the visible datamaps
-   */
-  List<TableIndex> getAllVisibleIndexes(CarbonTable carbonTable) throws IOException {
-    CarbonSessionInfo sessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo();
-    List<TableIndex> allDataMaps = getAllIndexes(carbonTable);
-    Iterator<TableIndex> dataMapIterator = allDataMaps.iterator();
-    while (dataMapIterator.hasNext()) {
-      TableIndex dataMap = dataMapIterator.next();
-      String dbName = carbonTable.getDatabaseName();
-      String tableName = carbonTable.getTableName();
-      String dmName = dataMap.getDataMapSchema().getDataMapName();
-      // TODO: need support get the visible status of datamap without sessionInfo in the future
-      if (sessionInfo != null) {
-        boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
-            String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
-                dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
-        if (!isDmVisible) {
-          LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
-              dmName, dbName, tableName));
-          dataMapIterator.remove();
-        }
-      } else {
-        String message = "Carbon session info is null";
-        LOGGER.info(message);
-      }
-    }
-    return allDataMaps;
-  }
-
-  /**
-   * It gives all indexes except the default index.
+   * It gives all indexes except the default index and secondary index.
+   * Collect's Coarse grain and Fine grain indexes on a table
    *
    * @return
    */
   public List<TableIndex> getAllIndexes(CarbonTable carbonTable) throws IOException {
-    List<DataMapSchema> dataMapSchemas = getDataMapSchemasOfTable(carbonTable);
+    String indexMeta = carbonTable.getTableInfo().getFactTable().getTableProperties()
+        .get(carbonTable.getCarbonTableIdentifier().getTableId());
+    IndexMetadata indexMetadata = IndexMetadata.deserialize(indexMeta);
     List<TableIndex> indexes = new ArrayList<>();
-    if (dataMapSchemas != null) {
-      for (DataMapSchema dataMapSchema : dataMapSchemas) {
-        RelationIdentifier identifier = dataMapSchema.getParentTables().get(0);
-        if (dataMapSchema.isIndex() && identifier.getTableId()
-            .equals(carbonTable.getTableId())) {
-          indexes.add(getIndex(carbonTable, dataMapSchema));
+    if (null != indexMetadata) {
+      // get bloom indexes and lucene indexes
+      for (Map.Entry<String, Map<String, Map<String, String>>> providerEntry : indexMetadata
+          .getIndexesMap().entrySet()) {
+        for (Map.Entry<String, Map<String, String>> indexEntry : providerEntry.getValue()
+            .entrySet()) {
+          if (!indexEntry.getValue().get(CarbonCommonConstants.INDEX_PROVIDER)
 
 Review comment:
   we are sending bloom and lucene to SI flow. But SI is still not like boom and lucene. So we are excluding here. This is causing multiple branches inside index again. we need to unify this

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-613516469
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2741/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-613269277
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1021/
   

----------------------------------------------------------------
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 #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on issue #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-611425237
 
 
   @ajantha-bhat But we are storing only index meta data to carbon table. So, all indexes info will be same

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405272670
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/index/IndexChooser.java
 ##########
 @@ -68,15 +67,14 @@
   public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
     // read all indexes for this table and populate CG and FG index list
-    List<TableIndex> visibleIndexes =
-        DataMapStoreManager.getInstance().getAllVisibleIndexes(carbonTable);
-    Map<String, DataMapStatusDetail> map = DataMapStatusManager.readDataMapStatusMap();
+    List<TableIndex> visibleIndexes = carbonTable.getAllVisibleIndexes();
     cgIndexes = new ArrayList<>(visibleIndexes.size());
     fgIndexes = new ArrayList<>(visibleIndexes.size());
     for (TableIndex visibleIndex : visibleIndexes) {
-      DataMapStatusDetail status = map.get(visibleIndex.getDataMapSchema().getDataMapName());
-      if (status != null && status.isEnabled()) {
-        IndexLevel level = visibleIndex.getIndexFactory().getDataMapLevel();
+      if (visibleIndex.getIndexSchema().getProperties().get(CarbonCommonConstants.INDEX_STATUS)
 
 Review comment:
   How do you handle the compatibility during upgrade ? 1.6 already wrote in system folder. Once upgraded. Table will not have these properties.

----------------------------------------------------------------
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 a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r406015062
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/ShowIndexesCommand.scala
 ##########
 @@ -54,69 +62,87 @@ case class ShowIndexesCommand(
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     val carbonTable = CarbonEnv.getCarbonTable(dbNameOp, tableName)(sparkSession)
     setAuditTable(carbonTable)
-    getFileIndexInfo(carbonTable) ++ getSIInfo(sparkSession, carbonTable)
-  }
-
-  // get info for 'index datamap'
-  private def getFileIndexInfo(carbonTable: CarbonTable): Seq[Row] = {
-    val indexes = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable).asScala
-    if (indexes != null && indexes.nonEmpty) {
-      indexes.map { index =>
-        Row(
-          index.getDataMapName,
-          index.getProviderName,
-          index.getIndexColumns.mkString(","),
-          index.getPropertiesAsString,
-          index.getStatus.name(),
-          index.getSyncStatus
-        )
-      }
-    } else {
-      Seq.empty
-    }
+    getIndexInfo(sparkSession, carbonTable)
   }
 
-  // get info for SI
-  private def getSIInfo(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
+  private def getIndexInfo(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
     CarbonInternalMetastore.refreshIndexInfo(
       carbonTable.getDatabaseName, tableName, carbonTable)(sparkSession)
-    val indexesMap = CarbonInternalScalaUtil.getIndexesMap(carbonTable)
-    if (null == indexesMap) {
-      throw new Exception("Secondary index information is not loaded in main table")
-    }
-    val indexTableMap = indexesMap.asScala
-    if (indexTableMap.nonEmpty) {
-      val indexList = indexTableMap.map { indexInfo =>
-        try {
-          val isSITableEnabled = sparkSession.sessionState.catalog
-            .getTableMetadata(TableIdentifier(indexInfo._1, dbNameOp)).storage.properties
-            .getOrElse("isSITableEnabled", "true").equalsIgnoreCase("true")
-          if (isSITableEnabled) {
-            (indexInfo._1, indexInfo._2.asScala.mkString(","), "enabled")
-          } else {
-            (indexInfo._1, indexInfo._2.asScala.mkString(","), "disabled")
+    val indexesMap = CarbonIndexUtil.getIndexesMap(carbonTable)
+    if (null != indexesMap) {
+      val indexTableMap = indexesMap.asScala
+      if (indexTableMap.nonEmpty) {
+        val secondaryIndex = indexTableMap.get(CarbonIndexProvider.SI.getIndexProviderName)
+        var finalIndexList: Seq[(String, String, String, String, String, String)] = Seq.empty
+
+        if (secondaryIndex.isDefined && null != secondaryIndex.get) {
+          val siIterator = secondaryIndex.get.entrySet().iterator()
+          while (siIterator.hasNext) {
+            val indexInfo = siIterator.next()
+            try {
+              val isSITableEnabled = sparkSession.sessionState.catalog
+                .getTableMetadata(TableIdentifier(indexInfo.getKey, dbNameOp)).storage.properties
+                .getOrElse("isSITableEnabled", "true").equalsIgnoreCase("true")
+              if (isSITableEnabled) {
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants.INDEX_COLUMNS), "NA", "enabled", "NA")
+              } else {
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants
+                                     .INDEX_COLUMNS), "NA", "disabled", "NA")
+              }
+            } catch {
+              case ex: Exception =>
+                LOGGER.error(s"Access storage properties from hive failed for index table: ${
+                  indexInfo.getKey
+                }")
+                finalIndexList = finalIndexList :+
+                                 (indexInfo.getKey, "carbondata", indexInfo.getValue
+                                   .get(CarbonCommonConstants.INDEX_COLUMNS), "NA", "UNKNOWN", "NA")
+            }
+          }
+        }
+
+        indexesMap.asScala
+          .filter(map => !map._1.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName))
 
 Review comment:
   Instead of 1 Map, if we keep 3 Map, it will occupy more memory right

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405278856
 
 

 ##########
 File path: format/src/main/thrift/schema.thrift
 ##########
 @@ -203,9 +203,9 @@ struct ParentColumnTableRelation {
    3: required string columnName
 }
 
-struct DataMapSchema  {
+struct IndexSchema  {
 
 Review comment:
   Changing thrift member name will leads to compatibility issue. Old store will have as object of DataMapSchema, type cast may fail. Have tried reading old store that has datamap schema ?

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


With regards,
Apache Git Services

[GitHub] [carbondata] asfgit closed pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688
 
 
   

----------------------------------------------------------------
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 #3688: [WIP] Refactor Index Metadata

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3688: [WIP] Refactor Index Metadata
URL: https://github.com/apache/carbondata/pull/3688#issuecomment-606703679
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2597/
   

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405999660
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonRefreshIndexCommand.scala
 ##########
 @@ -71,30 +80,81 @@ case class CarbonRefreshIndexCommand(
   private def refreshIndex(
       sparkSession: SparkSession,
       parentTable: CarbonTable,
-      indexOp: Optional[DataMapSchema]): Unit = {
-    val schema = indexOp.get
+      indexMetaData: IndexMetadata): Unit = {
+    var indexInfo: util.Map[String, String] = new util.HashMap[String, String]()
+    val allIndexesIterator = indexMetaData.getIndexesMap.entrySet().iterator()
+    breakable {
+      while (allIndexesIterator.hasNext) {
+        val currentIndex = allIndexesIterator.next()
+        if (!currentIndex.getKey.equalsIgnoreCase(CarbonIndexProvider.SI.getIndexProviderName)) {
+          val indexIterator = currentIndex.getValue.entrySet().iterator()
+          while (indexIterator.hasNext) {
+            val indexEntry = indexIterator.next()
+            if (indexEntry.getKey.equalsIgnoreCase(indexName)) {
+              indexInfo = indexEntry.getValue
+              break()
+            }
+          }
+        }
+      }
+    }
+    if (indexInfo.isEmpty) {
+      throw new MalformedIndexCommandException(
+        "Index with name `" + indexName + "` is not present" +
+        "on table `" + parentTable.getTableName + "`")
+    }
+    val indexProviderName = indexInfo.get(CarbonCommonConstants.INDEX_PROVIDER)
+    val schema = new DataMapSchema(indexName, indexProviderName)
+    schema.setProperties(indexInfo)
     if (!schema.isLazy) {
       throw new MalformedIndexCommandException(
         s"Non-lazy index $indexName does not support manual refresh")
     }
 
     val provider = DataMapManager.get().getDataMapProvider(parentTable, schema, sparkSession)
     provider.rebuild()
+    // enable bloom or lucene index
+    // get metadata lock to avoid concurrent create index operations
+    val metadataLock = CarbonLockFactory.getCarbonLockObj(
+      parentTable.getAbsoluteTableIdentifier,
+      LockUsage.METADATA_LOCK)
+    try {
+      if (metadataLock.lockWithRetries()) {
+        LOGGER.info(s"Acquired the metadata lock for table " +
+                    s"${ parentTable.getDatabaseName}.${ parentTable.getTableName }")
+        val oldIndexInfo = parentTable.getIndexInfo
+        val updatedIndexInfo = IndexTableInfo.enableIndex(oldIndexInfo, indexName)
+
+        // set index information in parent table
+        val parentIndexMetadata =
+          IndexMetadata.deserialize(parentTable.getTableInfo.getFactTable.getTableProperties
 
 Review comment:
   we already deserialize and store it in parent carbon table indexMeta right ? why not directly use it ?

----------------------------------------------------------------
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] ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3688: [CARBONDATA-3765] Refactor Index Metadata for CG and FG Indexes
URL: https://github.com/apache/carbondata/pull/3688#discussion_r405977323
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -101,49 +100,29 @@ private DataMapStoreManager() {
   }
 
   /**
-   * It only gives the visible datamaps
-   */
-  List<TableIndex> getAllVisibleIndexes(CarbonTable carbonTable) throws IOException {
-    CarbonSessionInfo sessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo();
-    List<TableIndex> allDataMaps = getAllIndexes(carbonTable);
-    Iterator<TableIndex> dataMapIterator = allDataMaps.iterator();
-    while (dataMapIterator.hasNext()) {
-      TableIndex dataMap = dataMapIterator.next();
-      String dbName = carbonTable.getDatabaseName();
-      String tableName = carbonTable.getTableName();
-      String dmName = dataMap.getDataMapSchema().getDataMapName();
-      // TODO: need support get the visible status of datamap without sessionInfo in the future
-      if (sessionInfo != null) {
-        boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
-            String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
-                dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
-        if (!isDmVisible) {
-          LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
-              dmName, dbName, tableName));
-          dataMapIterator.remove();
-        }
-      } else {
-        String message = "Carbon session info is null";
-        LOGGER.info(message);
-      }
-    }
-    return allDataMaps;
-  }
-
-  /**
-   * It gives all indexes except the default index.
+   * It gives all indexes except the default index and secondary index.
+   * Collect's Coarse grain and Fine grain indexes on a table
    *
    * @return
    */
   public List<TableIndex> getAllIndexes(CarbonTable carbonTable) throws IOException {
 
 Review comment:
   what is the point in calling this getAllIndex ?, if it is not giving all index (default and SI).
   may be call this get CGandFG index

----------------------------------------------------------------
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