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/25 06:49:28 UTC

[GitHub] [carbondata] vikramahuja1001 opened a new pull request #3678: [WIP]: index server concurrency fix

vikramahuja1001 opened a new pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678
 
 
    ### 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] kunal642 commented on a change in pull request #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#discussion_r398327064
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/SegmentProcessor.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * 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.datamap;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+
+public class SegmentProcessor implements Serializable {
+  private static HashMap<String, HashSet<SegmentWorkStatus>> globalWorkQueue =
 
 Review comment:
   Use ConcurrentHashMap

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603701974
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2552/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603703931
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/845/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-612932705
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1017/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-612903081
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2728/
   

----------------------------------------------------------------
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] kunal642 commented on a change in pull request #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#discussion_r398329624
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedCountRDD.scala
 ##########
 @@ -96,20 +99,78 @@ class DistributedCountRDD(@transient ss: SparkSession, dataMapFormat: Distributa
     new DistributedPruneRDD(ss, dataMapFormat).partitions
   }
 
-  private def generateFuture(split: Seq[InputSplit])
+  private def generateFuture(split: Seq[InputSplit], globalQueue: SegmentProcessor)
     (implicit executionContext: ExecutionContext) = {
     Future {
-      val segments = split.map { inputSplit =>
+
+      var segmentsWorkStatus = split.map { inputSplit =>
         val distributable = inputSplit.asInstanceOf[DataMapDistributableWrapper]
         distributable.getDistributable.getSegment
           .setReadCommittedScope(dataMapFormat.getReadCommittedScope)
-        distributable.getDistributable.getSegment
+
+        val processedSegments = globalQueue.ifProcessSegment(distributable.getDistributable
+          .getSegment.getSegmentNo, dataMapFormat.getCarbonTable.getTableId)
+
+        val segmentWorkStatusList = new SegmentWorkStatus(distributable.getDistributable
+          .getSegment, !processedSegments)
+
+        // if ifprocesssegment = true, iswaiting = false
+        val processedSegmentsList = globalQueue.processSegment(segmentWorkStatusList,
+          dataMapFormat.getCarbonTable.getTableId)
+        segmentWorkStatusList
+      }
+
+      val queueSize = globalQueue.queueSize()
+      val getGlobalworkQueue = globalQueue.getGlobalWorkQueue
+
+      var segmentsPositive: mutable.HashSet[SegmentWorkStatus] = mutable.HashSet.empty
+      var segmentsNegative: mutable.HashSet[SegmentWorkStatus] = mutable.HashSet.empty
+
+      segmentsWorkStatus.map { iter =>
+        if (iter.getWaiting == false) {
+          segmentsPositive.add(iter)
+        } else {
+          segmentsNegative.add(iter)
+        }
       }
+
       val defaultDataMap = DataMapStoreManager.getInstance
         .getDataMap(dataMapFormat.getCarbonTable, split.head
           .asInstanceOf[DataMapDistributableWrapper].getDistributable.getDataMapSchema)
-      defaultDataMap.getBlockRowCount(segments.toList.asJava, dataMapFormat
-        .getPartitions, defaultDataMap).asScala
+      var result = defaultDataMap.getBlockRowCount(segmentsPositive.map { iter =>
+        iter.getSegment }.toList.asJava, dataMapFormat.getPartitions,
+        defaultDataMap).asScala
+
+      //  delete from local
+      var segment = segmentsPositive.map { iter =>
+        globalQueue.updateWaitingStatus(iter, defaultDataMap.getTable.getTableId)
+      }
+
+      while (segmentsNegative != null && segmentsNegative.size != 0) {
+        segmentsWorkStatus = segmentsNegative.map { iter =>
+          val processedSegments = globalQueue.ifProcessSegment(iter.getSegment
+            .getSegmentNo, defaultDataMap.getTable.getTableId)
+          val processedSegmentsList = globalQueue.processSegment(iter,
+            dataMapFormat.getCarbonTable.getTableId)
+          iter
+        }.toSeq
+        segmentsPositive = mutable.HashSet.empty
+        segmentsNegative = mutable.HashSet.empty
+        segmentsWorkStatus.map { iter =>
+          if (iter.getWaiting == false) {
+            segmentsPositive.add(iter)
+          } else {
+            segmentsNegative.add(iter)
+          }
+        }
+        result = result.++(defaultDataMap.getBlockRowCount(segmentsPositive
 
 Review comment:
   Please remove all mutable collections and var. I dont see any real use for these

----------------------------------------------------------------
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] vikramahuja1001 commented on a change in pull request #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#discussion_r398585301
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedCountRDD.scala
 ##########
 @@ -96,20 +99,78 @@ class DistributedCountRDD(@transient ss: SparkSession, dataMapFormat: Distributa
     new DistributedPruneRDD(ss, dataMapFormat).partitions
   }
 
-  private def generateFuture(split: Seq[InputSplit])
+  private def generateFuture(split: Seq[InputSplit], globalQueue: SegmentProcessor)
     (implicit executionContext: ExecutionContext) = {
     Future {
-      val segments = split.map { inputSplit =>
+
+      var segmentsWorkStatus = split.map { inputSplit =>
         val distributable = inputSplit.asInstanceOf[DataMapDistributableWrapper]
         distributable.getDistributable.getSegment
           .setReadCommittedScope(dataMapFormat.getReadCommittedScope)
-        distributable.getDistributable.getSegment
+
+        val processedSegments = globalQueue.ifProcessSegment(distributable.getDistributable
+          .getSegment.getSegmentNo, dataMapFormat.getCarbonTable.getTableId)
+
+        val segmentWorkStatusList = new SegmentWorkStatus(distributable.getDistributable
+          .getSegment, !processedSegments)
+
+        // if ifprocesssegment = true, iswaiting = false
+        val processedSegmentsList = globalQueue.processSegment(segmentWorkStatusList,
+          dataMapFormat.getCarbonTable.getTableId)
+        segmentWorkStatusList
+      }
+
+      val queueSize = globalQueue.queueSize()
+      val getGlobalworkQueue = globalQueue.getGlobalWorkQueue
 
 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] kunal642 commented on a change in pull request #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#discussion_r398328095
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedCountRDD.scala
 ##########
 @@ -69,15 +70,17 @@ class DistributedCountRDD(@transient ss: SparkSession, dataMapFormat: Distributa
       DataMapStoreManager.getInstance().clearInvalidSegments(dataMapFormat.getCarbonTable,
         dataMapFormat.getInvalidSegments)
     }
+    val globalQueue = SegmentProcessor.getInstance()
     val futures = if (inputSplits.length <= numOfThreads) {
       inputSplits.map {
-        split => generateFuture(Seq(split))
+        split => generateFuture(Seq(split), globalQueue)
       }
     } else {
       DistributedRDDUtils.groupSplits(inputSplits, numOfThreads).map {
-        splits => generateFuture(splits)
+        splits => generateFuture(splits, globalQueue)
       }
     }
+    globalQueue.emptyQueue()
 
 Review comment:
   This will impact other jobs which might be still using globalQueue

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-612900566
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1015/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-604442311
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2568/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603750297
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/846/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-612932883
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2729/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-604350981
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2564/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-613006776
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2730/
   

----------------------------------------------------------------
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] vikramahuja1001 commented on a change in pull request #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#discussion_r398585393
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/SegmentProcessor.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * 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.datamap;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+
+public class SegmentProcessor implements Serializable {
+  private static HashMap<String, HashSet<SegmentWorkStatus>> globalWorkQueue =
 
 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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603750418
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2553/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603812825
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/849/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603813425
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2556/
   

----------------------------------------------------------------
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] kunal642 commented on a change in pull request #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#discussion_r398328476
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedCountRDD.scala
 ##########
 @@ -96,20 +99,78 @@ class DistributedCountRDD(@transient ss: SparkSession, dataMapFormat: Distributa
     new DistributedPruneRDD(ss, dataMapFormat).partitions
   }
 
-  private def generateFuture(split: Seq[InputSplit])
+  private def generateFuture(split: Seq[InputSplit], globalQueue: SegmentProcessor)
     (implicit executionContext: ExecutionContext) = {
     Future {
-      val segments = split.map { inputSplit =>
+
+      var segmentsWorkStatus = split.map { inputSplit =>
         val distributable = inputSplit.asInstanceOf[DataMapDistributableWrapper]
         distributable.getDistributable.getSegment
           .setReadCommittedScope(dataMapFormat.getReadCommittedScope)
-        distributable.getDistributable.getSegment
+
+        val processedSegments = globalQueue.ifProcessSegment(distributable.getDistributable
+          .getSegment.getSegmentNo, dataMapFormat.getCarbonTable.getTableId)
+
+        val segmentWorkStatusList = new SegmentWorkStatus(distributable.getDistributable
+          .getSegment, !processedSegments)
+
+        // if ifprocesssegment = true, iswaiting = false
+        val processedSegmentsList = globalQueue.processSegment(segmentWorkStatusList,
+          dataMapFormat.getCarbonTable.getTableId)
+        segmentWorkStatusList
+      }
+
+      val queueSize = globalQueue.queueSize()
+      val getGlobalworkQueue = globalQueue.getGlobalWorkQueue
 
 Review comment:
   Please remove all unused variables like 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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-612901063
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2727/
   

----------------------------------------------------------------
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] vikramahuja1001 commented on a change in pull request #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#discussion_r407479647
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedCountRDD.scala
 ##########
 @@ -96,20 +99,78 @@ class DistributedCountRDD(@transient ss: SparkSession, dataMapFormat: Distributa
     new DistributedPruneRDD(ss, dataMapFormat).partitions
   }
 
-  private def generateFuture(split: Seq[InputSplit])
+  private def generateFuture(split: Seq[InputSplit], globalQueue: SegmentProcessor)
     (implicit executionContext: ExecutionContext) = {
     Future {
-      val segments = split.map { inputSplit =>
+
+      var segmentsWorkStatus = split.map { inputSplit =>
         val distributable = inputSplit.asInstanceOf[DataMapDistributableWrapper]
         distributable.getDistributable.getSegment
           .setReadCommittedScope(dataMapFormat.getReadCommittedScope)
-        distributable.getDistributable.getSegment
+
+        val processedSegments = globalQueue.ifProcessSegment(distributable.getDistributable
+          .getSegment.getSegmentNo, dataMapFormat.getCarbonTable.getTableId)
+
+        val segmentWorkStatusList = new SegmentWorkStatus(distributable.getDistributable
+          .getSegment, !processedSegments)
+
+        // if ifprocesssegment = true, iswaiting = false
+        val processedSegmentsList = globalQueue.processSegment(segmentWorkStatusList,
+          dataMapFormat.getCarbonTable.getTableId)
+        segmentWorkStatusList
+      }
+
+      val queueSize = globalQueue.queueSize()
+      val getGlobalworkQueue = globalQueue.getGlobalWorkQueue
+
+      var segmentsPositive: mutable.HashSet[SegmentWorkStatus] = mutable.HashSet.empty
+      var segmentsNegative: mutable.HashSet[SegmentWorkStatus] = mutable.HashSet.empty
+
+      segmentsWorkStatus.map { iter =>
+        if (iter.getWaiting == false) {
+          segmentsPositive.add(iter)
+        } else {
+          segmentsNegative.add(iter)
+        }
       }
+
       val defaultDataMap = DataMapStoreManager.getInstance
         .getDataMap(dataMapFormat.getCarbonTable, split.head
           .asInstanceOf[DataMapDistributableWrapper].getDistributable.getDataMapSchema)
-      defaultDataMap.getBlockRowCount(segments.toList.asJava, dataMapFormat
-        .getPartitions, defaultDataMap).asScala
+      var result = defaultDataMap.getBlockRowCount(segmentsPositive.map { iter =>
+        iter.getSegment }.toList.asJava, dataMapFormat.getPartitions,
+        defaultDataMap).asScala
+
+      //  delete from local
+      var segment = segmentsPositive.map { iter =>
+        globalQueue.updateWaitingStatus(iter, defaultDataMap.getTable.getTableId)
+      }
+
+      while (segmentsNegative != null && segmentsNegative.size != 0) {
+        segmentsWorkStatus = segmentsNegative.map { iter =>
+          val processedSegments = globalQueue.ifProcessSegment(iter.getSegment
+            .getSegmentNo, defaultDataMap.getTable.getTableId)
+          val processedSegmentsList = globalQueue.processSegment(iter,
+            dataMapFormat.getCarbonTable.getTableId)
+          iter
+        }.toSeq
+        segmentsPositive = mutable.HashSet.empty
+        segmentsNegative = mutable.HashSet.empty
+        segmentsWorkStatus.map { iter =>
+          if (iter.getWaiting == false) {
+            segmentsPositive.add(iter)
+          } else {
+            segmentsNegative.add(iter)
+          }
+        }
+        result = result.++(defaultDataMap.getBlockRowCount(segmentsPositive
 
 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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603673692
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2550/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603680013
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2551/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603673353
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/843/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-604350260
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/856/
   

----------------------------------------------------------------
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] vikramahuja1001 commented on a change in pull request #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#discussion_r407479607
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/indexserver/DistributedCountRDD.scala
 ##########
 @@ -69,15 +70,17 @@ class DistributedCountRDD(@transient ss: SparkSession, dataMapFormat: Distributa
       DataMapStoreManager.getInstance().clearInvalidSegments(dataMapFormat.getCarbonTable,
         dataMapFormat.getInvalidSegments)
     }
+    val globalQueue = SegmentProcessor.getInstance()
     val futures = if (inputSplits.length <= numOfThreads) {
       inputSplits.map {
-        split => generateFuture(Seq(split))
+        split => generateFuture(Seq(split), globalQueue)
       }
     } else {
       DistributedRDDUtils.groupSplits(inputSplits, numOfThreads).map {
-        splits => generateFuture(splits)
+        splits => generateFuture(splits, globalQueue)
       }
     }
+    globalQueue.emptyQueue()
 
 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] vikramahuja1001 commented on issue #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-613253747
 
 
   @kunal642 , please 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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-612902649
 
 
   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1016/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-613012464
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1018/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-603679681
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/844/
   

----------------------------------------------------------------
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 #3678: [WIP]: index server concurrency fix

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3678: [WIP]: index server concurrency fix
URL: https://github.com/apache/carbondata/pull/3678#issuecomment-604444324
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/860/
   

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