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/12/02 14:32:48 UTC

[GitHub] [carbondata] marchpure commented on a change in pull request #4026: [CARBONDATA-4063] Refactor getBlockId and getShortBlockId functions

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
##########
@@ -635,44 +604,89 @@ public static String getCarbonMergeIndexExtension() {
     return MERGE_INDEX_FILE_EXT;
   }
 
+  /**
+   * Generate the blockId as per the block path
+   *
+   * @param identifier
+   * @param filePath
+   * @param segmentId
+   * @param isPartitionTable
+   * @return blockid, which is the identify of a block
+   */
+  public static String getBlockId(AbsoluteTableIdentifier identifier, String filePath,
+      String segmentId, boolean isPartitionTable) {
+    String blockName = filePath.substring(filePath.lastIndexOf(
+        CarbonCommonConstants.FILE_SEPARATOR) + 1);
+    String tablePath = identifier.getTablePath();
+
+    String partitionDir = "";
+    // 1. For block of Added Segments, The BlockId consistsof
+    //    <partitionPath><segmentId><blockName>
+    if (!filePath.startsWith(tablePath)) {
+      partitionDir = getPartitionDir(tablePath, filePath, blockName);
+      return partitionDir.replace(CarbonCommonConstants.FILE_SEPARATOR, "#")
+          + CarbonCommonConstants.FILE_SEPARATOR + segmentId
+          + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+    }
+
+    // 2. For block of partitiontable, The BlockId consistsof
+    //    <partitionPath><blockName>
+    if (isPartitionTable) {
+      partitionDir = getPartitionDir(tablePath, filePath, blockName);
+      return partitionDir.replace(CarbonCommonConstants.FILE_SEPARATOR, "#")
+          + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+    }
+
+    // 3. For nonpartitiontable, The BlockId consistsof
+    //    <segmentId><blockName>
+    return segmentId + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+  }
+
   /**
    * This method will remove strings in path and return short block id
    *
    * @param blockId
    * @return shortBlockId
    */
   public static String getShortBlockId(String blockId) {
-    String blockIdWithCompressorName =
-        blockId.replace(PARTITION_PREFIX + "0" + CarbonCommonConstants.FILE_SEPARATOR, "")
-            .replace(SEGMENT_PREFIX, "").replace(BATCH_PREFIX, CarbonCommonConstants.UNDERSCORE)
-            .replace(DATA_PART_PREFIX, "").replace(CARBON_DATA_EXT, "");
-    // to remove compressor name
-    if (!blockId.equalsIgnoreCase(blockIdWithCompressorName)) {
-      int index = blockIdWithCompressorName.lastIndexOf(POINT);
-      int fileSeperatorIndex = blockIdWithCompressorName.lastIndexOf(File.separator);
-      if (index != -1) {
-        String modifiedBlockId;
-        if (index > fileSeperatorIndex) {
-          // Default case when path ends with compressor name.
-          // Example: 0/0-0_0-0-0-1600789595862.snappy
-          modifiedBlockId =
-              blockIdWithCompressorName.replace(blockIdWithCompressorName.substring(index), "");
-        } else {
-          // in case of CACHE_LEVEL = BLOCKLET, blockId path contains both block id and blocklet id
-          // so check for next file seperator and remove compressor name.
-          // Example: 0/0-0_0-0-0-1600789595862.snappy/0
-          modifiedBlockId = blockIdWithCompressorName
-              .replace(blockIdWithCompressorName.substring(index, fileSeperatorIndex), "");
-        }
-        return modifiedBlockId;
-      } else {
-        return blockIdWithCompressorName;
-      }
-    } else {
-      return blockIdWithCompressorName;
+    // 1. If the blockid is already shortblockid, return directly
+    int suffixIndex = blockId.lastIndexOf(CARBON_DATA_EXT);
+    if (suffixIndex < 0) {
+      return blockId;
     }
+    // 2. get the filepath. in the type of part=a/part-0-0_batchno0-0-0-1597409791503.snappy
+    String filePath = blockId.substring(0, suffixIndex);
+
+    // 3. get the compressor name, in the type of '.snappy'/'.zstd'/'.gzip'
+    String compressorName = filePath.substring(filePath.lastIndexOf(POINT));
+
+    // 4. get rid of 'Part0/' 'Segment_' '_batchno' 'part-' '.carbondata' and compressorname
+    return blockId.replace(compressorName, "")
+        .replace(PARTITION_PREFIX + "0" + CarbonCommonConstants.FILE_SEPARATOR, "")
+        .replace(SEGMENT_PREFIX, "")
+        .replace(BATCH_PREFIX, CarbonCommonConstants.UNDERSCORE)
+        .replace(DATA_PART_PREFIX, "")
+        .replace(CARBON_DATA_EXT, "");
+  }
+
+  /**
+   * get the partition path in the block path
+   *
+   * @param tablePath
+   * @param filePath
+   * @param blockName
+   * @return blockid, which is the identify of a block
+   */
+  public static String getPartitionDir(String tablePath, String filePath, String blockName) {
+    // The filepath is consist with <tablePath><partitionPath><blockName>
+    // The partitionPath is the string truncated between tablePath and blockName
+    if (!filePath.startsWith(tablePath)) {
+      return filePath.substring(0, filePath.length() - blockName.length());
+    }
+    return filePath.substring(tablePath.length() + 1, filePath.length() - blockName.length() - 1);
   }
 
+

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