[GitHub] [carbondata] ShreelekhyaG opened a new pull request #3988: [WIP] Clean index files when clean files command executed

classic Classic list List threaded Threaded
171 messages Options
1234567 ... 9
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox

ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534677231



##########
File path: core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
##########
@@ -672,7 +678,7 @@ public static boolean isMaxQueryTimeoutExceeded(long fileTimestamp) {
 
     long minutesElapsed = (difference / (1000 * 60));
 
-    return minutesElapsed > maxTime;
+    return minutesElapsed >= maxTime;

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
##########
@@ -2558,8 +2558,11 @@ public static long getCarbonIndexSize(SegmentFileStore fileStore,
   // Get the total size of carbon data and the total size of carbon index
   public static HashMap<String, Long> getDataSizeAndIndexSize(String tablePath,
       Segment segment) throws IOException {
+    SegmentFileStore fileStore = null;
     if (segment.getSegmentFileName() != null) {
-      SegmentFileStore fileStore = new SegmentFileStore(tablePath, segment.getSegmentFileName());
+      fileStore = new SegmentFileStore(tablePath, segment.getSegmentFileName());
+    }
+    if (segment.getSegmentFileName() != null && fileStore.getSegmentFile() != null) {

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] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534677505



##########
File path: core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
##########
@@ -241,13 +266,17 @@ public String writeMergeIndexFileBasedOnSegmentFile(String segmentId,
           break;
         }
       }
+      if (!table.isIndexTable()) {

Review comment:
       done, maintained mapping for SI and MT now.

##########
File path: core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
##########
@@ -277,21 +305,18 @@ public String writeMergeIndexFileBasedOnSegmentFile(String segmentId,
         LOGGER.error("unable to write segment file during merge index writing: " + ex.getMessage());
         throw ex;
       }
-      boolean status = SegmentFileStore.updateTableStatusFile(table, segmentId, newSegmentFileName,
-          table.getCarbonTableIdentifier().getTableId(), segmentFileStore);
-      if (!status) {
-        // revert to original segment file as the table status update has failed.
-        SegmentStatusManager.writeStringIntoFile(path, content);
-        // delete merge index file.
-        for (String file : mergeIndexFiles) {
-          FileFactory.getCarbonFile(file).delete();
-        }
-        // no need to delete index files, so return from here.
-        return uuid;
-      }
     }
-    for (CarbonFile file : indexFiles) {
-      file.delete();
+    boolean status = SegmentFileStore.updateTableStatusFile(table, segmentId, newSegmentFileName,

Review comment:
       `mergeIndexBasedOnSegmentFile `is called when old store is used (here, already segment detail is loaded as success in tablestatus), it will modify size details and segment file name.




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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534677732



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -267,15 +269,20 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
         // Merge all partition files into a single file.
         segmentFileName =
           mergedLoadNumber + "_" + carbonLoadModel.getFactTimeStamp
-        val segmentFile = SegmentFileStore
-          .mergeSegmentFiles(readPath,
-            segmentFileName,
-            CarbonTablePath.getSegmentFilesLocation(carbonLoadModel.getTablePath))
-        if (segmentFile != null) {
-          SegmentFileStore
-            .moveFromTempFolder(segmentFile,
-              carbonLoadModel.getFactTimeStamp + ".tmp",
-              carbonLoadModel.getTablePath)
+        if (!isPropertySet(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
+          CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT)) {

Review comment:
       Done

##########
File path: integration/spark/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
##########
@@ -167,6 +173,21 @@ object CarbonMergeFilesRDD {
             executorService.submit(new Runnable {
               override def run(): Unit = {
                 ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
+                // If Alter merge index for old tables is triggered, do not delete index files
+                // immediately to avoid index file not found during concurrent queries
+                if (readFileFooterFromCarbonDataFile ||
+                    !isPropertySet(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
+                      CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT)) {

Review comment:
       Done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala
##########
@@ -226,85 +233,59 @@ object SecondaryIndexUtil {
               validSegmentsToUse.toList.asJava,
               indexCarbonTable)
           }
-          mergedSegments.asScala.map { seg =>
-            val file = SegmentFileStore.writeSegmentFile(
-              indexCarbonTable,
-              seg.getLoadName,
-              carbonLoadModel.getFactTimeStamp.toString,
-              null,
-              null)
-            val segment = new Segment(seg.getLoadName, file)
-            SegmentFileStore.updateTableStatusFile(indexCarbonTable,
-              seg.getLoadName,
-              file,
-              indexCarbonTable.getCarbonTableIdentifier.getTableId,
-              new SegmentFileStore(tablePath, segment.getSegmentFileName))
-            segment
-          }
-
-          val statusLock =
-            new SegmentStatusManager(indexCarbonTable.getAbsoluteTableIdentifier).getTableStatusLock
-          try {
-            val retryCount = CarbonLockUtil.getLockProperty(CarbonCommonConstants
-              .NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
-              CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
-            val maxTimeout = CarbonLockUtil.getLockProperty(CarbonCommonConstants
-              .MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
-              CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
-            if (statusLock.lockWithRetries(retryCount, maxTimeout)) {
-              val endTime = System.currentTimeMillis()
-              val loadMetadataDetails = SegmentStatusManager
-                .readLoadMetadata(indexCarbonTable.getMetadataPath)
-              loadMetadataDetails.foreach(loadMetadataDetail => {
-                if (rebuiltSegments.contains(loadMetadataDetail.getLoadName)) {
-                  loadMetadataDetail.setLoadStartTime(carbonLoadModel.getFactTimeStamp)
-                  loadMetadataDetail.setLoadEndTime(endTime)
-                  CarbonLoaderUtil
-                    .addDataIndexSizeIntoMetaEntry(loadMetadataDetail,
-                      loadMetadataDetail.getLoadName,
-                      indexCarbonTable)
-                }
-              })
-              SegmentStatusManager
-                .writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(tablePath),
-                  loadMetadataDetails)
-            } else {
-              throw new RuntimeException(
-                "Not able to acquire the lock for table status updation for table " + databaseName +
-                "." + indexCarbonTable.getTableName)
-            }
-          } finally {
-            if (statusLock != null) {
-              statusLock.unlock()
-            }
-          }
-          // clear the indexSchema cache for the merged segments, as the index files and
-          // data files are rewritten after compaction
+          val segmentToLoadStartTimeMap: scala.collection.mutable.Map[String, java.lang.Long] =
+            scala.collection.mutable.Map()
           if (mergedSegments.size > 0) {
-
-            // merge index files for merged segments
-            CarbonMergeFilesRDD.mergeIndexFiles(sc.sparkSession,
-              rebuiltSegments.toSeq,
-              segmentIdToLoadStartTimeMap,
-              indexCarbonTable.getTablePath,
-              indexCarbonTable, mergeIndexProperty = false
-            )
-
-            if (CarbonProperties.getInstance()
-              .isDistributedPruningEnabled(indexCarbonTable.getDatabaseName,
-                indexCarbonTable.getTableName)) {
-              try {
-                IndexServer.getClient
-                  .invalidateSegmentCache(indexCarbonTable,
-                    rebuiltSegments.toArray,
-                    SparkSQLUtil.getTaskGroupId(sc.sparkSession))
-              } catch {
-                case _: Exception =>
+            // merge index files and write segment file for merged segments
+            mergedSegments.asScala.map { seg =>
+              if (isPropertySet(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
+                CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT)) {
+                new CarbonIndexFileMergeWriter(indexCarbonTable).mergeCarbonIndexFilesOfSegment(seg
+                  .getLoadName,
+                  tablePath,
+                  false,
+                  carbonLoadModel.getFactTimeStamp.toString)
+              }
+              val file = SegmentFileStore.writeSegmentFile(
+                indexCarbonTable,
+                seg.getLoadName,
+                carbonLoadModel.getFactTimeStamp.toString,
+                null,
+                null)
+              segmentToLoadStartTimeMap.put(seg.getLoadName,
+                carbonLoadModel.getFactTimeStamp)
+              // clear the indexSchema cache for the merged segments, as the index files and
+              // data files are rewritten after compaction
+              if (CarbonProperties.getInstance()
+                .isDistributedPruningEnabled(indexCarbonTable.getDatabaseName,
+                  indexCarbonTable.getTableName)) {
+                try {
+                  IndexServer.getClient
+                    .invalidateSegmentCache(indexCarbonTable,
+                      rebuiltSegments.toArray,
+                      SparkSQLUtil.getTaskGroupId(sc.sparkSession))
+                } catch {
+                  case _: Exception =>
+                }
               }
+              val segment = new Segment(seg.getLoadName, file)
+              segment
+            }
+            if (compactionType == null) {

Review comment:
       Done

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
##########
@@ -530,21 +530,22 @@ class CarbonIndexFileMergeTestCase
       FileFactory.getCarbonFile(table.getAbsoluteTableIdentifier.getTablePath)
         .listFiles(true, new CarbonFileFilter {
           override def accept(file: CarbonFile): Boolean = {
-            file.getName.endsWith(extension)

Review comment:
       Done

##########
File path: processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
##########
@@ -1221,6 +1223,20 @@ public static String mergeIndexFilesInPartitionedSegment(CarbonTable table, Stri
             tempFolderPath, currPartitionSpec);
   }
 
+  public static String mergeIndexFilesInTempSegment(CarbonTable table, String segmentId,
+      String segmentPath, String uuid) {
+    try {
+      return new CarbonIndexFileMergeWriter(table)
+          .writeMergeIndexFileBasedOnSegmentFolder(null, false, segmentPath, segmentId, uuid,
+              false);
+    } catch (IOException e) {
+      String message =
+          "Failed to merge index files in path: " + segmentPath + ". " + e.getMessage();

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] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534678076



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala
##########
@@ -226,85 +233,59 @@ object SecondaryIndexUtil {
               validSegmentsToUse.toList.asJava,
               indexCarbonTable)
           }
-          mergedSegments.asScala.map { seg =>
-            val file = SegmentFileStore.writeSegmentFile(
-              indexCarbonTable,
-              seg.getLoadName,
-              carbonLoadModel.getFactTimeStamp.toString,
-              null,
-              null)
-            val segment = new Segment(seg.getLoadName, file)
-            SegmentFileStore.updateTableStatusFile(indexCarbonTable,
-              seg.getLoadName,
-              file,
-              indexCarbonTable.getCarbonTableIdentifier.getTableId,
-              new SegmentFileStore(tablePath, segment.getSegmentFileName))
-            segment
-          }
-
-          val statusLock =
-            new SegmentStatusManager(indexCarbonTable.getAbsoluteTableIdentifier).getTableStatusLock
-          try {
-            val retryCount = CarbonLockUtil.getLockProperty(CarbonCommonConstants
-              .NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
-              CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
-            val maxTimeout = CarbonLockUtil.getLockProperty(CarbonCommonConstants
-              .MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
-              CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
-            if (statusLock.lockWithRetries(retryCount, maxTimeout)) {
-              val endTime = System.currentTimeMillis()
-              val loadMetadataDetails = SegmentStatusManager
-                .readLoadMetadata(indexCarbonTable.getMetadataPath)
-              loadMetadataDetails.foreach(loadMetadataDetail => {
-                if (rebuiltSegments.contains(loadMetadataDetail.getLoadName)) {
-                  loadMetadataDetail.setLoadStartTime(carbonLoadModel.getFactTimeStamp)
-                  loadMetadataDetail.setLoadEndTime(endTime)
-                  CarbonLoaderUtil
-                    .addDataIndexSizeIntoMetaEntry(loadMetadataDetail,
-                      loadMetadataDetail.getLoadName,
-                      indexCarbonTable)
-                }
-              })
-              SegmentStatusManager
-                .writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(tablePath),
-                  loadMetadataDetails)
-            } else {
-              throw new RuntimeException(
-                "Not able to acquire the lock for table status updation for table " + databaseName +
-                "." + indexCarbonTable.getTableName)
-            }
-          } finally {
-            if (statusLock != null) {
-              statusLock.unlock()
-            }
-          }
-          // clear the indexSchema cache for the merged segments, as the index files and
-          // data files are rewritten after compaction
+          val segmentToLoadStartTimeMap: scala.collection.mutable.Map[String, java.lang.Long] =
+            scala.collection.mutable.Map()
           if (mergedSegments.size > 0) {

Review comment:
       removed the 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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534676791



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##########
@@ -162,15 +162,25 @@ public static void writeSegmentFile(String tablePath, String segmentId, String t
    * corresponding partitions.
    */
   public static void writeSegmentFile(String tablePath, final String taskNo, String location,
-      String timeStamp, List<String> partitionNames, boolean isMergeIndexFlow) throws IOException {
-    String tempFolderLoc = timeStamp + ".tmp";
-    String writePath = CarbonTablePath.getSegmentFilesLocation(tablePath) + "/" + tempFolderLoc;
+      String timeStamp, List<String> partitionNames, boolean isMergeIndexFlow,
+      boolean readFileFooterFromCarbonDataFile) throws IOException {
+    String tempFolderLoc;
+    String writePath;
+    if (!readFileFooterFromCarbonDataFile) {
+      tempFolderLoc = timeStamp + ".tmp";
+      writePath = CarbonTablePath.getSegmentFilesLocation(tablePath) + "/" + tempFolderLoc;
+    } else {
+      // If Alter merge index for old tables is triggered,
+      // directly write mergeindex file into segment file location
+      tempFolderLoc = location;
+      writePath = CarbonTablePath.getSegmentFilesLocation(tablePath);
+    }
     CarbonFile carbonFile = FileFactory.getCarbonFile(writePath);
     if (!carbonFile.exists()) {
       carbonFile.mkdirs();
     }
     CarbonFile tempFolder;
-    if (isMergeIndexFlow) {
+    if (isMergeIndexFlow || readFileFooterFromCarbonDataFile) {

Review comment:
       Done. removed unnecessary code from this method.




----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-737695440


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-737700791


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


----------------------------------------------------------------
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] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534763831



##########
File path: core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##########
@@ -439,6 +430,73 @@ public boolean accept(CarbonFile file) {
     return null;
   }
 
+  /**
+   * Get old and invalid files which have already been merged to a mergeindex file.In segment folder
+   * we may have both .index files and .mergeindex files, as we are not deleting index files
+   * immediately for old tables, this method reads mergeindex file and adds mapped index files to a
+   * list and returns.If more than one mergeindex file is present, considers the latest one as valid
+   * Ex: We have 3 files in segment. Segment0/ 1.index , 1.mergeindex file, 1.carbondata.
+   * 1.index is merged to 1.mergeindex. Here it returns merged index file - 1.index.
+   */
+  public static Set<String> getInvalidAndMergedIndexFiles(List<String> indexFiles)
+      throws IOException {
+    SegmentIndexFileStore indexFileStore = new SegmentIndexFileStore();

Review comment:
       This method is called during read, when segment/table status file is not present or gets deleted, and when writing segment data size in tablestatus file. So for all old/new tables this method is called and taken care of. For new tables also, when stale data is present, it is used to filter invalid files. Ex: SI load when MT has stale index files, as during SI load MT segment file/ status file name is not updated, we directly get from segment directory.




----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-737784784


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-737786983


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-744438607


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-744439091


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-746418688


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-746420767


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-749503878


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-749504289


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-749562847


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


----------------------------------------------------------------
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] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-749568290


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


----------------------------------------------------------------
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] QiangCai commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

QiangCai commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-762090072


   how about skip carbonindex files step to generate carbonindexmerge files directly on driver side?


----------------------------------------------------------------
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 #3988: [CARBONDATA-4037] Improve the table status and segment file writing

GitBox
In reply to this post by GitBox

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


   > how about skip carbonindex files step to generate carbonindexmerge files directly on driver side?
   
   
   
   @QiangCai : Can you explain more about this ? you mean don't read index file to make merge index file. Just concurrently (by acquiring lock) write all index info to single merge index ?


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

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


1234567 ... 9