[GitHub] [carbondata] Kejian-Li opened a new pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

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

[GitHub] [carbondata] Kejian-Li opened a new pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox

Kejian-Li opened a new pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947


    ### Why is this PR needed?
    When some segments's status is INSERT IN PROGRESS, update(or delete) of whole table is not allowed.
    But the fact is that, update(or delete)'s result can be confirmed to be correct when there is insert operations executed in the meanwhile. We shall allow update(or delete) and insert to be executed concurrently.
   
    ### What changes were proposed in this PR?
    Improve IUD Concurrency by removing the condition: "if there are insertinprogress segments in tablestatus, make update fail directly." Some testcases are added to confirm the result of concurrent insert and update.
       
    ### Does this PR introduce any user interface change?
    - No
   
    ### Is any new testcase added?
    - 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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox

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


   Can one of the admins verify this patch?


----------------------------------------------------------------
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] jackylk commented on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

jackylk commented on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-696672614


   add to whitelist


----------------------------------------------------------------
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 #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

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






----------------------------------------------------------------
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 #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

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


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


----------------------------------------------------------------
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 #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

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


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


----------------------------------------------------------------
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] Zhangshunyu commented on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

Zhangshunyu commented on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-697074445


   pls handle the filed testcases


----------------------------------------------------------------
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] Zhangshunyu removed a comment on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

Zhangshunyu removed a comment on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-697074445


   pls handle the filed testcases


----------------------------------------------------------------
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] Zhangshunyu commented on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

Zhangshunyu commented on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-697074579


   please handle the failed testcases


----------------------------------------------------------------
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] Zhangshunyu commented on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

Zhangshunyu commented on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-697078961


   Consider a scenario that user want to insert data and then update the table, the dml command is in time series which is sent 1 by 1, but the 2nd(update) cmd is sent by another driver and start to execute while the 1st(insert) is still in running, so it comes to the concurrent scenario described in this pr.
   If we allow update table who has segment with status 'INSERT_INPROGRESS', the 2nd cmd(update) will be executed successfully and the data inserted by the 1st cmd would not updated by the 2nd cmd.
   Maybe in this scenario the result is not what the user expected.


----------------------------------------------------------------
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] jackylk commented on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

jackylk commented on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-696672614


   add to whitelist


----------------------------------------------------------------
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 #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

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






----------------------------------------------------------------
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] Zhangshunyu commented on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

Zhangshunyu commented on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-697074445






----------------------------------------------------------------
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] Zhangshunyu removed a comment on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

Zhangshunyu removed a comment on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-697074445


   pls handle the filed testcases


----------------------------------------------------------------
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] Zhangshunyu edited a comment on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

Zhangshunyu edited a comment on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-697078961


   Consider a scenario that the user want to insert data and then update the table, the dml commands are sent one by one in time series, but the 2nd(update) cmd is sent by another driver and start to execute while the 1st(insert) is still  running, therefore the concurrency scenario described in this PR appears.
   If we allow update table who has segment with status 'INSERT_INPROGRESS', the 2nd cmd(update) will execute successfully and the data inserted by the 1st cmd would not updated by the 2nd cmd.
   In this case, the result may not be what the user expected.


----------------------------------------------------------------
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] Zhangshunyu edited a comment on pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

Zhangshunyu edited a comment on pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#issuecomment-697074579


   please handle the failed test cases


----------------------------------------------------------------
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] marchpure commented on a change in pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

marchpure commented on a change in pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#discussion_r494819647



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatus.java
##########
@@ -31,6 +31,12 @@
   @SerializedName("Success")
   SUCCESS("Success"),
 
+  /**

Review comment:
       revert this change

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -918,17 +893,40 @@ public static Boolean isCompactionInProgress(CarbonTable carbonTable) {
     return compactionInProgress;
   }
 
+  /**
+   * Return true if insert or insert overwrite is in progress for specified table
+   */
+  public static Boolean isInsertInProgress(CarbonTable carbonTable) {
+    if (carbonTable == null) {
+      return false;
+    }
+    boolean loadInProgress = false;
+    String metaPath = carbonTable.getMetadataPath();
+    LoadMetadataDetails[] listOfLoadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(metaPath);
+    if (listOfLoadFolderDetailsArray.length != 0) {
+      for (LoadMetadataDetails loadDetail :listOfLoadFolderDetailsArray) {
+        SegmentStatus segmentStatus = loadDetail.getSegmentStatus();
+        if (segmentStatus == SegmentStatus.INSERT_IN_PROGRESS
+        || segmentStatus == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS) {
+          loadInProgress =
+                  isLoadInProgress(carbonTable.getAbsoluteTableIdentifier(),
+                          loadDetail.getLoadName());
+        }
+      }
+    }
+    return loadInProgress;
+  }
+
   /**
    * Return true if insert overwrite is in progress for specified table
    */
-  public static Boolean isOverwriteInProgressInTable(CarbonTable carbonTable) {
+  public static boolean isInsertOverwriteInProgress(CarbonTable carbonTable) {
     if (carbonTable == null) {
       return false;
     }
     boolean loadInProgress = false;
     String metaPath = carbonTable.getMetadataPath();
-    LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-        SegmentStatusManager.readLoadMetadata(metaPath);
+    LoadMetadataDetails[] listOfLoadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(metaPath);

Review comment:
       revert this change

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -918,17 +893,40 @@ public static Boolean isCompactionInProgress(CarbonTable carbonTable) {
     return compactionInProgress;
   }
 
+  /**
+   * Return true if insert or insert overwrite is in progress for specified table
+   */
+  public static Boolean isInsertInProgress(CarbonTable carbonTable) {

Review comment:
       revert this change

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/DeleteFromTableEventListener.scala
##########
@@ -53,6 +53,12 @@ class DeleteFromTableEventListener extends OperationEventListener with Logging {
               carbonTable
                 .getDatabaseName
             }.${ carbonTable.getTableName }]")
+        } else if (!carbonTable.getIndexesMap.isEmpty) {

Review comment:
       change to
   "if (carbonTable.isIndexTable || !carbonTable.getIndexesMap.isEmpty)"

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/UpdateTablePreEventListener.scala
##########
@@ -49,7 +49,6 @@ class UpdateTablePreEventListener extends OperationEventListener with Logging {
               carbonTable
                 .getDatabaseName
             }.${ carbonTable.getTableName }]. Drop all indexes and retry")

Review comment:
       revert this change

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
##########
@@ -68,8 +53,8 @@ class TestInsertAndOtherCommandConcurrent extends QueryTest with BeforeAndAfterA
       .mode(SaveMode.Overwrite)
       .save()
 
-    sql(s"insert into orders select * from temp_table")
-    sql(s"insert into orders_overwrite select * from temp_table")
+    sql(s"insert into orders select * from temp_table") // load_0   success

Review comment:
       revert this change

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
##########
@@ -18,21 +18,18 @@ package org.apache.carbondata.spark.testsuite.iud
 
 import java.io.File
 
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.test.SparkTestQueryExecutor
-import org.apache.spark.sql.test.util.QueryTest
-import org.apache.spark.sql.{CarbonEnv, Row, SaveMode}
-import org.scalatest.BeforeAndAfterAll
-
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.index.Segment
 import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.spark.sql.hive.CarbonRelation

Review comment:
       revert this change

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
##########
@@ -68,8 +53,8 @@ class TestInsertAndOtherCommandConcurrent extends QueryTest with BeforeAndAfterA
       .mode(SaveMode.Overwrite)
       .save()
 
-    sql(s"insert into orders select * from temp_table")
-    sql(s"insert into orders_overwrite select * from temp_table")
+    sql(s"insert into orders select * from temp_table") // load_0   success
+    sql(s"insert into orders_overwrite select * from temp_table") // load_0   success

Review comment:
       revert this change

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
##########
@@ -92,54 +77,46 @@ class TestInsertAndOtherCommandConcurrent extends QueryTest with BeforeAndAfterA
   private def createTable(tableName: String, schema: StructType): Unit = {
     val schemaString = schema.fields.map(x => x.name + " " + x.dataType.typeName).mkString(", ")
     sql(s"CREATE TABLE $tableName ($schemaString) stored as carbondata tblproperties" +
-        s"('sort_scope'='local_sort','sort_columns'='o_country,o_name,o_phonetype,o_serialname," +
-        s"o_comment')")
-  }
-
-  override def afterAll {
-    executorService.shutdownNow()
-    dropTable()
+      s"('sort_scope'='local_sort','sort_columns'='o_country,o_name,o_phonetype,o_serialname," +

Review comment:
       revert this change

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/view/timeseries/TestCreateMVWithTimeSeries.scala
##########
@@ -122,22 +139,23 @@ class TestCreateMVWithTimeSeries extends QueryTest with BeforeAndAfterAll {
       sql("drop materialized view if exists mv4")
       sql("drop materialized view if exists mv5")
     }
+
     dropMVs
     sql(
       "create materialized view mv1 as " +
-      "select timeseries(projectjoindate,'second'), sum(projectcode) from maintable group by timeseries(projectjoindate,'second')")
+        "select timeseries(projectjoindate,'second'), sum(projectcode) from maintable group by timeseries(projectjoindate,'second')")
     sql(
       "create materialized view mv2 as " +
-      "select timeseries(projectjoindate,'hour'), sum(projectcode) from maintable group by timeseries(projectjoindate,'hour')")
+        "select timeseries(projectjoindate,'hour'), sum(projectcode) from maintable group by timeseries(projectjoindate,'hour')")

Review comment:
       revert all format change

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
##########
@@ -170,17 +146,19 @@ class TestInsertAndOtherCommandConcurrent extends QueryTest with BeforeAndAfterA
   }
 
   test("alter rename table should fail if insert overwrite is in progress") {
-    val future = runSqlAsync("insert overwrite table orders select * from orders_overwrite")
+    sql("drop table if exists other_orders")
+    val future = AsyncExecutorUtils.runSqlAsync("insert overwrite table orders select * from orders_overwrite")
     val ex = intercept[ConcurrentOperationException] {
-      sql("alter table orders rename to other")
+      sql("alter table orders rename to other_orders")

Review comment:
       other_orders => different orders




----------------------------------------------------------------
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] marchpure commented on a change in pull request #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

marchpure commented on a change in pull request #3947:
URL: https://github.com/apache/carbondata/pull/3947#discussion_r494842615



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
##########
@@ -65,7 +65,7 @@ private[sql] case class CarbonProjectForDeleteCommand(
         s"Unsupported delete operation on table containing mixed format segments")
     }
 
-    if (SegmentStatusManager.isLoadInProgressInTable(carbonTable)) {
+    if (SegmentStatusManager.isInsertOverwriteInProgress(carbonTable)) {
       throw new ConcurrentOperationException(carbonTable, "loading", "data delete")

Review comment:
       loading->insert overwrite

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
##########
@@ -18,34 +18,18 @@
 package org.apache.carbondata.spark.testsuite.iud
 
 import java.text.SimpleDateFormat
-import java.util
-import java.util.concurrent.{Callable, ExecutorService, Executors, Future}
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.fs.Path
-import org.apache.spark.sql.test.util.QueryTest
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.sql.{DataFrame, SaveMode}
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.block.SegmentProperties
-import org.apache.carbondata.core.datastore.page.ColumnPage
 import org.apache.carbondata.core.exception.ConcurrentOperationException
-import org.apache.carbondata.core.features.TableOperation
-import org.apache.carbondata.core.index.dev.cgindex.{CoarseGrainIndex, CoarseGrainIndexFactory}
-import org.apache.carbondata.core.index.dev.{IndexBuilder, IndexWriter}
-import org.apache.carbondata.core.index.{IndexInputSplit, IndexMeta, Segment}
-import org.apache.carbondata.core.indexstore.PartitionSpec
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, IndexSchema}
-import org.apache.carbondata.core.scan.filter.intf.ExpressionType
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.events.Event
+import org.apache.spark.sql.test.util.QueryTest
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{DataFrame, Row, SaveMode}
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}

Review comment:
       revert this change

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/AsyncExecutorUtils.scala
##########
@@ -0,0 +1,200 @@
+
+
+
+package org.apache.carbondata.spark.testsuite.iud

Review comment:
       remove into util package




----------------------------------------------------------------
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 #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

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


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


----------------------------------------------------------------
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 #3947: [CARBONDATA-4003] Improve IUD Concurrency

GitBox
In reply to this post by GitBox

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


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


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


1234 ... 6