[GitHub] [carbondata] ajantha-bhat opened a new pull request #3787: [WIP] support sort_scope for index creation

classic Classic list List threaded Threaded
54 messages Options
123
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3787: [WIP][CARBONDATA-3923] support global sort for SI

GitBox

CarbonDataQA1 commented on pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#issuecomment-689696689


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2285/
   


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#issuecomment-689992523


   @QiangCai , @akashrn5 @kunal642 : please start review. Complex type testcase and some more testcases can be added. I will add in 2 hours


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

akashrn5 commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486083368



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -801,6 +802,26 @@ object CommonUtil {
     }
   }
 
+  def validateGlobalSortPartitions(propertiesMap: mutable.Map[String, String]): Unit = {
+    if (propertiesMap.get("global_sort_partitions").isDefined) {

Review comment:
       replace with `propertiesMap.contains("global_sort_partitions")`

##########
File path: docs/index/secondary-index-guide.md
##########
@@ -84,7 +84,8 @@ EXPLAIN SELECT a from maintable where c = 'cd';
   'carbondata'
   PROPERTIES('table_blocksize'='1')
   ```
-
+  **NOTE**:
+  * supported properties are table_blocksize, column_meta_cache, cache_level, carbon.column.compressor, sort_scope, global_sort_partitions

Review comment:
       ```suggestion
     * supported properties are table_blocksize, column_meta_cache, cache_level, carbon.column.compressor, sort_scope and global_sort_partitions.
   ```

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/index/CarbonIndexUtil.scala
##########
@@ -297,6 +298,10 @@ object CarbonIndexUtil {
       segmentIdToLoadStartTimeMapping = scala.collection.mutable
         .Map((carbonLoadModel.getSegmentId, carbonLoadModel.getFactTimeStamp))
     }
+    val indexCarbonTable = CarbonEnv.getCarbonTable(

Review comment:
       index table object is already present, please remove this

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference
+                val proj = indexCarbonTable.getCreateOrderColumn

Review comment:
       rename to projections

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -428,4 +552,40 @@ object SecondaryIndexCreator {
     }
     threadPoolSize
   }
+
+  def dataFrameOfSegments(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      projections: String,
+      segments: Array[String]): DataFrame = {
+    try {
+      CarbonUtils
+        .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+                   carbonTable.getDatabaseName + CarbonCommonConstants.POINT +
+                   carbonTable.getTableName,
+          segments.mkString(","))
+      val logicalPlan = sparkSession
+        .sql(s"select $projections from ${ carbonTable.getDatabaseName }.${
+          carbonTable
+            .getTableName
+        }")
+        .queryExecution
+        .logical
+      val positionId = UnresolvedAlias(Alias(UnresolvedFunction("getPositionId",
+        Seq.empty, isDistinct = false), "positionId")())

Review comment:
       use constant for `positionId` from CarbonCommonConstant

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -801,6 +802,26 @@ object CommonUtil {
     }
   }
 
+  def validateGlobalSortPartitions(propertiesMap: mutable.Map[String, String]): Unit = {
+    if (propertiesMap.get("global_sort_partitions").isDefined) {
+      val globalSortPartitionsProp = propertiesMap("global_sort_partitions")
+      var pass = false
+      try {
+        val globalSortPartitions = Integer.parseInt(globalSortPartitionsProp)
+        if (globalSortPartitions > 0) {
+          pass = true
+        }
+      } catch {
+        case _ =>
+      }
+      if (!pass) {

Review comment:
       no need of this pass variable, just try to parse to string with `.toInt` in try block, catch the `NumberFormatException` in catch block and throw the below exception with message. Like store this message in a variable, `if (globalSortPartitions > 0)` check can be after the catch block, and throw that same error message.

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -801,6 +802,26 @@ object CommonUtil {
     }
   }
 
+  def validateGlobalSortPartitions(propertiesMap: mutable.Map[String, String]): Unit = {
+    if (propertiesMap.get("global_sort_partitions").isDefined) {
+      val globalSortPartitionsProp = propertiesMap("global_sort_partitions")
+      var pass = false
+      try {
+        val globalSortPartitions = Integer.parseInt(globalSortPartitionsProp)

Review comment:
       `Integer.parseInt(globalSortPartitionsProp)` to `globalSortPartitionsProp.toInt`

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
##########
@@ -81,6 +81,22 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
   def apply(plan: LogicalPlan): Seq[SparkPlan] = {
     val transformedPlan = makeDeterministic(plan)
     transformedPlan match {
+      case GlobalLimit(IntegerLiteral(limit),
+      LocalLimit(IntegerLiteral(limitValue),
+      p@PhysicalOperation(projects, filters, l: LogicalRelation))) if l
+        .relation

Review comment:
       move to above line

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference

Review comment:
       can you edit comment in  more meaningful sentence ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference
+                val proj = indexCarbonTable.getCreateOrderColumn
+                  .asScala
+                  .map(_.getColName)
+                  .filterNot(_.equals("positionReference")).toSet
+                val explodeColumn = mainTable.getCreateOrderColumn.asScala
+                  .filter(x => x.getDataType.isComplexType &&
+                               proj.contains(x.getColName))
+                var dataFrame = dataFrameOfSegments(sc.sparkSession,
+                  mainTable,
+                  proj.mkString(","),
+                  Array(eachSegment))
+                // flatten the complex SI
+                if (explodeColumn.nonEmpty) {
+                  val columns = dataFrame.schema.map { x =>
+                    if (x.name.equals(explodeColumn.head.getColName)) {
+                      functions.explode_outer(functions.col(x.name))
+                    } else {
+                      functions.col(x.name)
+                    }
+                  }
+                  dataFrame = dataFrame.select(columns: _*)
+                }
+                val dataLoadSchema = new CarbonDataLoadSchema(indexCarbonTable)
+                carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+                carbonLoadModel.setTableName(indexCarbonTable.getTableName)
+                carbonLoadModel.setDatabaseName(indexCarbonTable.getDatabaseName)
+                carbonLoadModel.setTablePath(indexCarbonTable.getTablePath)
+                carbonLoadModel.setFactTimeStamp(secondaryIndexModel
+                  .segmentIdToLoadStartTimeMapping(eachSegment))
+                carbonLoadModel.setSegmentId(eachSegment)
+                var result: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
+                try {
+                  val configuration = FileFactory.getConfiguration
+                  configuration.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, eachSegment)
+                  def findCarbonScanRDD(rdd: RDD[_]): Unit = {
+                    rdd match {
+                      case d: CarbonScanRDD[_] =>

Review comment:
       ```suggestion
                         case carbonScanRDD: CarbonScanRDD[_] =>
   ```

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -428,4 +552,40 @@ object SecondaryIndexCreator {
     }
     threadPoolSize
   }
+
+  def dataFrameOfSegments(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      projections: String,
+      segments: Array[String]): DataFrame = {
+    try {
+      CarbonUtils
+        .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+                   carbonTable.getDatabaseName + CarbonCommonConstants.POINT +
+                   carbonTable.getTableName,
+          segments.mkString(","))

Review comment:
       move this to above line

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -428,4 +552,40 @@ object SecondaryIndexCreator {
     }
     threadPoolSize
   }
+
+  def dataFrameOfSegments(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      projections: String,
+      segments: Array[String]): DataFrame = {
+    try {
+      CarbonUtils
+        .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+                   carbonTable.getDatabaseName + CarbonCommonConstants.POINT +
+                   carbonTable.getTableName,
+          segments.mkString(","))
+      val logicalPlan = sparkSession
+        .sql(s"select $projections from ${ carbonTable.getDatabaseName }.${
+          carbonTable
+            .getTableName
+        }")

Review comment:
       move 570 and 571 to 569

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference
+                val proj = indexCarbonTable.getCreateOrderColumn
+                  .asScala
+                  .map(_.getColName)
+                  .filterNot(_.equals("positionReference")).toSet

Review comment:
       use constant `POSITION_REFERENCE`

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
##########
@@ -95,6 +95,8 @@ class CarbonScanRDD[T: ClassTag](
 
   private var readCommittedScope: ReadCommittedScope = _
 
+  private var validateSegmentToAccess: Boolean = true

Review comment:
       please add description to variable, why it is added and in which case it has to be used

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference
+                val proj = indexCarbonTable.getCreateOrderColumn
+                  .asScala
+                  .map(_.getColName)
+                  .filterNot(_.equals("positionReference")).toSet
+                val explodeColumn = mainTable.getCreateOrderColumn.asScala
+                  .filter(x => x.getDataType.isComplexType &&
+                               proj.contains(x.getColName))
+                var dataFrame = dataFrameOfSegments(sc.sparkSession,
+                  mainTable,
+                  proj.mkString(","),
+                  Array(eachSegment))
+                // flatten the complex SI
+                if (explodeColumn.nonEmpty) {
+                  val columns = dataFrame.schema.map { x =>
+                    if (x.name.equals(explodeColumn.head.getColName)) {
+                      functions.explode_outer(functions.col(x.name))
+                    } else {
+                      functions.col(x.name)
+                    }
+                  }
+                  dataFrame = dataFrame.select(columns: _*)
+                }
+                val dataLoadSchema = new CarbonDataLoadSchema(indexCarbonTable)
+                carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+                carbonLoadModel.setTableName(indexCarbonTable.getTableName)
+                carbonLoadModel.setDatabaseName(indexCarbonTable.getDatabaseName)
+                carbonLoadModel.setTablePath(indexCarbonTable.getTablePath)
+                carbonLoadModel.setFactTimeStamp(secondaryIndexModel
+                  .segmentIdToLoadStartTimeMapping(eachSegment))
+                carbonLoadModel.setSegmentId(eachSegment)
+                var result: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
+                try {
+                  val configuration = FileFactory.getConfiguration
+                  configuration.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, eachSegment)
+                  def findCarbonScanRDD(rdd: RDD[_]): Unit = {
+                    rdd match {
+                      case d: CarbonScanRDD[_] =>
+                        d.setValidateSegmentToAccess(false)
+                      case others =>
+                        others.dependencies.foreach {x => findCarbonScanRDD(x.rdd)}
+                    }
+                  }
+                  findCarbonScanRDD(dataFrame.rdd)
+                  // accumulator to collect segment metadata
+                  val segmentMetaDataAccumulator = sc.sparkSession.sqlContext
+                    .sparkContext
+                    .collectionAccumulator[Map[String, SegmentMetaDataInfo]]
+                  // TODO: use new insert into flow, instead of DataFrame prepare RDD[InternalRow]
+                  result = DataLoadProcessBuilderOnSpark.loadDataUsingGlobalSort(
+                    sc.sparkSession,
+                    Some(dataFrame),
+                    carbonLoadModel,
+                    hadoopConf = configuration, segmentMetaDataAccumulator)
+                }
+                SegmentFileStore
+                  .writeSegmentFile(indexCarbonTable,
+                    eachSegment,
+                    String.valueOf(carbonLoadModel.getFactTimeStamp))
+                segmentToLoadStartTimeMap
+                  .put(eachSegment,
+                    String.valueOf(carbonLoadModel.getFactTimeStamp))

Review comment:
       move this line above




----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#issuecomment-690103617


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2294/
   


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#issuecomment-690105564


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/4033/
   


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486196569



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -801,6 +802,26 @@ object CommonUtil {
     }
   }
 
+  def validateGlobalSortPartitions(propertiesMap: mutable.Map[String, String]): Unit = {
+    if (propertiesMap.get("global_sort_partitions").isDefined) {
+      val globalSortPartitionsProp = propertiesMap("global_sort_partitions")
+      var pass = false
+      try {
+        val globalSortPartitions = Integer.parseInt(globalSortPartitionsProp)
+        if (globalSortPartitions > 0) {
+          pass = true
+        }
+      } catch {
+        case _ =>
+      }
+      if (!pass) {

Review comment:
       keeping a flag and handling error at one place is better I feel




----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486204559



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -428,4 +552,40 @@ object SecondaryIndexCreator {
     }
     threadPoolSize
   }
+
+  def dataFrameOfSegments(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      projections: String,
+      segments: Array[String]): DataFrame = {
+    try {
+      CarbonUtils
+        .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+                   carbonTable.getDatabaseName + CarbonCommonConstants.POINT +
+                   carbonTable.getTableName,
+          segments.mkString(","))
+      val logicalPlan = sparkSession
+        .sql(s"select $projections from ${ carbonTable.getDatabaseName }.${
+          carbonTable
+            .getTableName
+        }")

Review comment:
       Moved. These are created by reformat command itself (ctrl + alt + shift + L), so need the correct tool to properly reformat or not 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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486217971



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
##########
@@ -95,6 +95,8 @@ class CarbonScanRDD[T: ClassTag](
 
   private var readCommittedScope: ReadCommittedScope = _
 
+  private var validateSegmentToAccess: Boolean = true

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486218311



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference
+                val proj = indexCarbonTable.getCreateOrderColumn
+                  .asScala
+                  .map(_.getColName)
+                  .filterNot(_.equals("positionReference")).toSet
+                val explodeColumn = mainTable.getCreateOrderColumn.asScala
+                  .filter(x => x.getDataType.isComplexType &&
+                               proj.contains(x.getColName))
+                var dataFrame = dataFrameOfSegments(sc.sparkSession,
+                  mainTable,
+                  proj.mkString(","),
+                  Array(eachSegment))
+                // flatten the complex SI
+                if (explodeColumn.nonEmpty) {
+                  val columns = dataFrame.schema.map { x =>
+                    if (x.name.equals(explodeColumn.head.getColName)) {
+                      functions.explode_outer(functions.col(x.name))
+                    } else {
+                      functions.col(x.name)
+                    }
+                  }
+                  dataFrame = dataFrame.select(columns: _*)
+                }
+                val dataLoadSchema = new CarbonDataLoadSchema(indexCarbonTable)
+                carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+                carbonLoadModel.setTableName(indexCarbonTable.getTableName)
+                carbonLoadModel.setDatabaseName(indexCarbonTable.getDatabaseName)
+                carbonLoadModel.setTablePath(indexCarbonTable.getTablePath)
+                carbonLoadModel.setFactTimeStamp(secondaryIndexModel
+                  .segmentIdToLoadStartTimeMapping(eachSegment))
+                carbonLoadModel.setSegmentId(eachSegment)
+                var result: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
+                try {
+                  val configuration = FileFactory.getConfiguration
+                  configuration.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, eachSegment)
+                  def findCarbonScanRDD(rdd: RDD[_]): Unit = {
+                    rdd match {
+                      case d: CarbonScanRDD[_] =>
+                        d.setValidateSegmentToAccess(false)
+                      case others =>
+                        others.dependencies.foreach {x => findCarbonScanRDD(x.rdd)}
+                    }
+                  }
+                  findCarbonScanRDD(dataFrame.rdd)
+                  // accumulator to collect segment metadata
+                  val segmentMetaDataAccumulator = sc.sparkSession.sqlContext
+                    .sparkContext
+                    .collectionAccumulator[Map[String, SegmentMetaDataInfo]]
+                  // TODO: use new insert into flow, instead of DataFrame prepare RDD[InternalRow]
+                  result = DataLoadProcessBuilderOnSpark.loadDataUsingGlobalSort(
+                    sc.sparkSession,
+                    Some(dataFrame),
+                    carbonLoadModel,
+                    hadoopConf = configuration, segmentMetaDataAccumulator)
+                }
+                SegmentFileStore
+                  .writeSegmentFile(indexCarbonTable,
+                    eachSegment,
+                    String.valueOf(carbonLoadModel.getFactTimeStamp))
+                segmentToLoadStartTimeMap
+                  .put(eachSegment,
+                    String.valueOf(carbonLoadModel.getFactTimeStamp))

Review comment:
       Moved. These are created by reformat command itself (ctrl + alt + shift + L), so need the correct tool to properly reformat or not 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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486218379



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference
+                val proj = indexCarbonTable.getCreateOrderColumn
+                  .asScala
+                  .map(_.getColName)
+                  .filterNot(_.equals("positionReference")).toSet
+                val explodeColumn = mainTable.getCreateOrderColumn.asScala
+                  .filter(x => x.getDataType.isComplexType &&
+                               proj.contains(x.getColName))
+                var dataFrame = dataFrameOfSegments(sc.sparkSession,
+                  mainTable,
+                  proj.mkString(","),
+                  Array(eachSegment))
+                // flatten the complex SI
+                if (explodeColumn.nonEmpty) {
+                  val columns = dataFrame.schema.map { x =>
+                    if (x.name.equals(explodeColumn.head.getColName)) {
+                      functions.explode_outer(functions.col(x.name))
+                    } else {
+                      functions.col(x.name)
+                    }
+                  }
+                  dataFrame = dataFrame.select(columns: _*)
+                }
+                val dataLoadSchema = new CarbonDataLoadSchema(indexCarbonTable)
+                carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+                carbonLoadModel.setTableName(indexCarbonTable.getTableName)
+                carbonLoadModel.setDatabaseName(indexCarbonTable.getDatabaseName)
+                carbonLoadModel.setTablePath(indexCarbonTable.getTablePath)
+                carbonLoadModel.setFactTimeStamp(secondaryIndexModel
+                  .segmentIdToLoadStartTimeMapping(eachSegment))
+                carbonLoadModel.setSegmentId(eachSegment)
+                var result: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
+                try {
+                  val configuration = FileFactory.getConfiguration
+                  configuration.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, eachSegment)
+                  def findCarbonScanRDD(rdd: RDD[_]): Unit = {
+                    rdd match {
+                      case d: CarbonScanRDD[_] =>

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -428,4 +552,40 @@ object SecondaryIndexCreator {
     }
     threadPoolSize
   }
+
+  def dataFrameOfSegments(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      projections: String,
+      segments: Array[String]): DataFrame = {
+    try {
+      CarbonUtils
+        .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+                   carbonTable.getDatabaseName + CarbonCommonConstants.POINT +
+                   carbonTable.getTableName,
+          segments.mkString(","))
+      val logicalPlan = sparkSession
+        .sql(s"select $projections from ${ carbonTable.getDatabaseName }.${
+          carbonTable
+            .getTableName
+        }")
+        .queryExecution
+        .logical
+      val positionId = UnresolvedAlias(Alias(UnresolvedFunction("getPositionId",
+        Seq.empty, isDistinct = false), "positionId")())

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
##########
@@ -81,6 +81,22 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
   def apply(plan: LogicalPlan): Seq[SparkPlan] = {
     val transformedPlan = makeDeterministic(plan)
     transformedPlan match {
+      case GlobalLimit(IntegerLiteral(limit),
+      LocalLimit(IntegerLiteral(limitValue),
+      p@PhysicalOperation(projects, filters, l: LogicalRelation))) if l
+        .relation

Review comment:
       Moved. These are created by reformat command itself (ctrl + alt + shift + L), so need the correct tool to properly reformat or not 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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486218662



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/index/CarbonIndexUtil.scala
##########
@@ -297,6 +298,10 @@ object CarbonIndexUtil {
       segmentIdToLoadStartTimeMapping = scala.collection.mutable
         .Map((carbonLoadModel.getSegmentId, carbonLoadModel.getFactTimeStamp))
     }
+    val indexCarbonTable = CarbonEnv.getCarbonTable(

Review comment:
       ok.done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference
+                val proj = indexCarbonTable.getCreateOrderColumn

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486218886



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -428,4 +552,40 @@ object SecondaryIndexCreator {
     }
     threadPoolSize
   }
+
+  def dataFrameOfSegments(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      projections: String,
+      segments: Array[String]): DataFrame = {
+    try {
+      CarbonUtils
+        .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+                   carbonTable.getDatabaseName + CarbonCommonConstants.POINT +
+                   carbonTable.getTableName,
+          segments.mkString(","))

Review comment:
       Moved. These are created by reformat command itself (ctrl + alt + shift + L), so need the correct tool to properly reformat or not use it.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference
+                val proj = indexCarbonTable.getCreateOrderColumn
+                  .asScala
+                  .map(_.getColName)
+                  .filterNot(_.equals("positionReference")).toSet

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -152,68 +158,181 @@ object SecondaryIndexCreator {
           LOGGER.info("spark.dynamicAllocation.maxExecutors property is set to =" + execInstance)
         }
       }
-      var futureObjectList = List[java.util.concurrent.Future[Array[(String, Boolean)]]]()
-      for (eachSegment <- validSegmentList) {
-        val segId = eachSegment
-        futureObjectList :+= executorService.submit(new Callable[Array[(String, Boolean)]] {
-          @throws(classOf[Exception])
-          override def call(): Array[(String, Boolean)] = {
-            ThreadLocalSessionInfo.getOrCreateCarbonSessionInfo().getNonSerializableExtraInfo
-              .put("carbonConf", SparkSQLUtil.sessionState(sc.sparkSession).newHadoopConf())
-            var eachSegmentSecondaryIndexCreationStatus: Array[(String, Boolean)] = Array.empty
-            CarbonLoaderUtil.checkAndCreateCarbonDataLocation(segId, indexCarbonTable)
-            val carbonLoadModel = getCopyObject(secondaryIndexModel)
-            carbonLoadModel
-              .setFactTimeStamp(secondaryIndexModel.segmentIdToLoadStartTimeMapping(eachSegment))
-            carbonLoadModel.setTablePath(secondaryIndexModel.carbonTable.getTablePath)
-            val secondaryIndexCreationStatus = new CarbonSecondaryIndexRDD(sc.sparkSession,
-              new SecondaryIndexCreationResultImpl,
-              carbonLoadModel,
-              secondaryIndexModel.secondaryIndex,
-              segId, execInstance, indexCarbonTable, forceAccessSegment, isCompactionCall).collect()
+      var successSISegments: List[String] = List()
+      var failedSISegments: List[String] = List()
+      val sort_scope = indexCarbonTable.getTableInfo.getFactTable.getTableProperties
+        .get("sort_scope")
+      if (sort_scope != null && sort_scope.equalsIgnoreCase("global_sort")) {
+        val mainTable = secondaryIndexModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        var futureObjectList = List[java.util.concurrent.Future[Array[(String,
+          (LoadMetadataDetails, ExecutionErrors))]]]()
+        for (eachSegment <- validSegmentList) {
+          futureObjectList :+= executorService
+            .submit(new Callable[Array[(String, (LoadMetadataDetails, ExecutionErrors))]] {
+              @throws(classOf[Exception])
+              override def call(): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+                val carbonLoadModel = getCopyObject(secondaryIndexModel)
+                // loading, we need to query main table add position reference

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486219088



##########
File path: docs/index/secondary-index-guide.md
##########
@@ -84,7 +84,8 @@ EXPLAIN SELECT a from maintable where c = 'cd';
   'carbondata'
   PROPERTIES('table_blocksize'='1')
   ```
-
+  **NOTE**:
+  * supported properties are table_blocksize, column_meta_cache, cache_level, carbon.column.compressor, sort_scope, global_sort_partitions

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486219230



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -801,6 +802,26 @@ object CommonUtil {
     }
   }
 
+  def validateGlobalSortPartitions(propertiesMap: mutable.Map[String, String]): Unit = {
+    if (propertiesMap.get("global_sort_partitions").isDefined) {

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -801,6 +802,26 @@ object CommonUtil {
     }
   }
 
+  def validateGlobalSortPartitions(propertiesMap: mutable.Map[String, String]): Unit = {
+    if (propertiesMap.get("global_sort_partitions").isDefined) {
+      val globalSortPartitionsProp = propertiesMap("global_sort_partitions")
+      var pass = false
+      try {
+        val globalSortPartitions = Integer.parseInt(globalSortPartitionsProp)

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#issuecomment-690129651


   @akashrn5 : handled comments, please check and merge once build passes


----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

akashrn5 commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486228364



##########
File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondryIndex.scala
##########
@@ -86,6 +86,64 @@ class TestSIWithSecondryIndex extends QueryTest with BeforeAndAfterAll {
       .contains("Alter table drop column operation failed:"))
   }
 
+  test("test create secondary index global sort after insert") {
+    sql("drop table if exists table1")
+    sql("create table table1 (name string, id string, country string) stored as carbondata")
+    sql("insert into table1 select 'xx', '2', 'china' union all select 'xx', '1', 'india'")
+    sql("create index table1_index on table table1(id, country) as 'carbondata' properties" +
+        "('sort_scope'='global_sort', 'Global_sort_partitions'='3')")
+    checkAnswerWithoutSort(sql("select id, country from table1_index"),
+      Seq(Row("1", "india"), Row("2", "china")))
+    // check for valid sort_scope
+    checkExistence(sql("describe formatted table1_index"), true, "Sort Scope global_sort")
+    // check the invalid sort scope
+    assert(intercept[MalformedCarbonCommandException](sql(
+      "create index index_2 on table table1(id, country) as 'carbondata' properties" +
+      "('sort_scope'='tim_sort', 'Global_sort_partitions'='3')"))
+      .getMessage
+      .contains("Invalid SORT_SCOPE tim_sort"))
+    // check for invalid global_sort_partitions
+    assert(intercept[MalformedCarbonCommandException](sql(
+      "create index index_2 on table table1(id, country) as 'carbondata' properties" +
+      "('sort_scope'='global_sort', 'Global_sort_partitions'='-1')"))
+      .getMessage
+      .contains("Table property global_sort_partitions : -1 is invalid"))
+    sql("drop index table1_index on table1")

Review comment:
       can just do drop table, it will drop index too, no need to separately run drop index and suggest to give a better tableName and index name and please check other test for same input.




----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] kunal642 commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

kunal642 commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486243458



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -801,6 +802,26 @@ object CommonUtil {
     }
   }
 
+  def validateGlobalSortPartitions(propertiesMap: mutable.Map[String, String]): Unit = {
+    if (propertiesMap.get("global_sort_partitions").isDefined) {
+      val globalSortPartitionsProp = propertiesMap("global_sort_partitions")
+      var pass = false
+      try {
+        val globalSortPartitions = Integer.parseInt(globalSortPartitionsProp)
+        if (globalSortPartitions > 0) {
+          pass = true
+        }
+      } catch {
+        case _ =>
+      }
+      if (!pass) {

Review comment:
       no, keeping this variable doesn't make sense. please catch Parsing Exception an throw MalformedCarbonCommandException direclty




----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] kunal642 commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

kunal642 commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486246357



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
##########
@@ -89,7 +104,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
         try {
           pruneFilterProject(
             l,
-            projects.filterNot(_.name.equalsIgnoreCase(CarbonCommonConstants.POSITION_ID)),

Review comment:
       why is the filter removed?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486261767



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -801,6 +802,26 @@ object CommonUtil {
     }
   }
 
+  def validateGlobalSortPartitions(propertiesMap: mutable.Map[String, String]): Unit = {
+    if (propertiesMap.get("global_sort_partitions").isDefined) {
+      val globalSortPartitionsProp = propertiesMap("global_sort_partitions")
+      var pass = false
+      try {
+        val globalSortPartitions = Integer.parseInt(globalSortPartitionsProp)
+        if (globalSortPartitions > 0) {
+          pass = true
+        }
+      } catch {
+        case _ =>
+      }
+      if (!pass) {

Review comment:
       If there is only one condition I would have done that only, I need to check `globalSortPartitions > 0` also and throw same error. Hence handing error at once place




----------------------------------------------------------------
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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3787: [CARBONDATA-3923] support global sort for SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3787:
URL: https://github.com/apache/carbondata/pull/3787#discussion_r486264018



##########
File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondryIndex.scala
##########
@@ -86,6 +86,64 @@ class TestSIWithSecondryIndex extends QueryTest with BeforeAndAfterAll {
       .contains("Alter table drop column operation failed:"))
   }
 
+  test("test create secondary index global sort after insert") {
+    sql("drop table if exists table1")
+    sql("create table table1 (name string, id string, country string) stored as carbondata")
+    sql("insert into table1 select 'xx', '2', 'china' union all select 'xx', '1', 'india'")
+    sql("create index table1_index on table table1(id, country) as 'carbondata' properties" +
+        "('sort_scope'='global_sort', 'Global_sort_partitions'='3')")
+    checkAnswerWithoutSort(sql("select id, country from table1_index"),
+      Seq(Row("1", "india"), Row("2", "china")))
+    // check for valid sort_scope
+    checkExistence(sql("describe formatted table1_index"), true, "Sort Scope global_sort")
+    // check the invalid sort scope
+    assert(intercept[MalformedCarbonCommandException](sql(
+      "create index index_2 on table table1(id, country) as 'carbondata' properties" +
+      "('sort_scope'='tim_sort', 'Global_sort_partitions'='3')"))
+      .getMessage
+      .contains("Invalid SORT_SCOPE tim_sort"))
+    // check for invalid global_sort_partitions
+    assert(intercept[MalformedCarbonCommandException](sql(
+      "create index index_2 on table table1(id, country) as 'carbondata' properties" +
+      "('sort_scope'='global_sort', 'Global_sort_partitions'='-1')"))
+      .getMessage
+      .contains("Table property global_sort_partitions : -1 is invalid"))
+    sql("drop index table1_index on table1")

Review comment:
       a) I know, but calling drop index will not add extra overhead.
   b) For test cases these names are enough ! This is not an example 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:
[hidden email]


123