You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/17 09:10:34 UTC

[GitHub] [incubator-pinot] siddharthteotia opened a new pull request #5711: Prototype for supporting inverted index drop from v3 format

siddharthteotia opened a new pull request #5711:
URL: https://github.com/apache/incubator-pinot/pull/5711


   POC code for supporting inverted index removal and file reorg. Standalone code which is not yet wired with SegmentPreprocessor.
   
    


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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] kishoreg commented on a change in pull request #5711: [WIP] - Supporting inverted index drop from v3 format

Posted by GitBox <gi...@apache.org>.
kishoreg commented on a change in pull request #5711:
URL: https://github.com/apache/incubator-pinot/pull/5711#discussion_r458572366



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/store/SingleFileIndexDirectory.java
##########
@@ -339,9 +344,92 @@ public void close()
 
   @Override
   public void removeIndex(String columnName, ColumnIndexType indexType) {
-    throw new UnsupportedOperationException(
-        "Index removal is not supported for single file index format. Requested colum: " + columnName + " indexType: "
-            + indexType);
+    if (indexType != ColumnIndexType.INVERTED_INDEX) {
+      throw new UnsupportedOperationException("Currently only inverted index removal is supported");
+    }
+    IndexKey indexKey = new IndexKey(columnName, indexType);
+    if (!columnEntries.containsKey(indexKey)) {
+      throw new IllegalStateException("Attempting to drop inverted index for non-existent column");
+    }
+    IndexEntry indexEntry = columnEntries.get(indexKey);
+    long indexStartOffset = indexEntry.startOffset;
+    SortedMap<Long, IndexEntry> indexStartMap = new TreeMap<>();
+    for (Map.Entry<IndexKey, IndexEntry> columnEntry : columnEntries.entrySet()) {
+      long startOffset = columnEntry.getValue().startOffset;
+      if (startOffset > indexStartOffset) {
+        indexStartMap.put(startOffset, columnEntry.getValue());
+      }
+    }
+    long runningSize = 0;
+    List<Long> offsetAccum = new ArrayList<>();
+    try {
+      long copyToOffset = indexStartOffset;
+      for (Map.Entry<Long, IndexEntry> offsetEntry : indexStartMap.entrySet()) {
+        IndexEntry entry = offsetEntry.getValue();
+        runningSize += entry.size;
+        if (runningSize >= MAX_ALLOCATION_SIZE) {
+          copyAndRemap(indexStartMap, offsetAccum, offsetEntry.getKey(), copyToOffset);
+          copyToOffset += runningSize - entry.size;
+          runningSize = entry.size;
+          offsetAccum.clear();
+        }
+        offsetAccum.add(offsetEntry.getKey());
+      }
+      if (offsetAccum.size() > 0) {
+        copyAndRemap(indexStartMap, offsetAccum, offsetAccum.get(0) + runningSize, copyToOffset);
+        copyToOffset += runningSize;
+      }
+      columnEntries.remove(indexKey);
+      RandomAccessFile randomFile = new RandomAccessFile(indexFile, "rw");
+      randomFile.setLength(copyToOffset);
+      updateIndexMap();
+    } catch (Exception e) {
+      // throw exception
+    }
+  }
+
+  private void copyAndRemap(SortedMap<Long, IndexEntry> startOffsets, List<Long> offsetAccum,

Review comment:
       smart idea!

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/store/SingleFileIndexDirectory.java
##########
@@ -339,9 +344,92 @@ public void close()
 
   @Override
   public void removeIndex(String columnName, ColumnIndexType indexType) {
-    throw new UnsupportedOperationException(
-        "Index removal is not supported for single file index format. Requested colum: " + columnName + " indexType: "
-            + indexType);
+    if (indexType != ColumnIndexType.INVERTED_INDEX) {

Review comment:
       why limit to INVERTED_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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] siddharthteotia commented on pull request #5711: [WIP] - Supporting inverted index drop from v3 format

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on pull request #5711:
URL: https://github.com/apache/incubator-pinot/pull/5711#issuecomment-832971753


   @Jackie-Jiang , @kishoreg
   
   Will resume work on 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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5711: [WIP] - Supporting inverted index drop from v3 format

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on a change in pull request #5711:
URL: https://github.com/apache/incubator-pinot/pull/5711#discussion_r459480938



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/store/SingleFileIndexDirectory.java
##########
@@ -339,9 +344,92 @@ public void close()
 
   @Override
   public void removeIndex(String columnName, ColumnIndexType indexType) {
-    throw new UnsupportedOperationException(
-        "Index removal is not supported for single file index format. Requested colum: " + columnName + " indexType: "
-            + indexType);
+    if (indexType != ColumnIndexType.INVERTED_INDEX) {

Review comment:
       Yeah, we could refactor/restructure such that it becomes a generic repackaging of indexes (e.g. drop any column/index, add new (??))

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/store/SingleFileIndexDirectory.java
##########
@@ -339,9 +344,92 @@ public void close()
 
   @Override
   public void removeIndex(String columnName, ColumnIndexType indexType) {
-    throw new UnsupportedOperationException(
-        "Index removal is not supported for single file index format. Requested colum: " + columnName + " indexType: "
-            + indexType);
+    if (indexType != ColumnIndexType.INVERTED_INDEX) {
+      throw new UnsupportedOperationException("Currently only inverted index removal is supported");
+    }
+    IndexKey indexKey = new IndexKey(columnName, indexType);
+    if (!columnEntries.containsKey(indexKey)) {
+      throw new IllegalStateException("Attempting to drop inverted index for non-existent column");
+    }
+    IndexEntry indexEntry = columnEntries.get(indexKey);
+    long indexStartOffset = indexEntry.startOffset;
+    SortedMap<Long, IndexEntry> indexStartMap = new TreeMap<>();
+    for (Map.Entry<IndexKey, IndexEntry> columnEntry : columnEntries.entrySet()) {
+      long startOffset = columnEntry.getValue().startOffset;
+      if (startOffset > indexStartOffset) {
+        indexStartMap.put(startOffset, columnEntry.getValue());
+      }
+    }
+    long runningSize = 0;
+    List<Long> offsetAccum = new ArrayList<>();
+    try {
+      long copyToOffset = indexStartOffset;
+      for (Map.Entry<Long, IndexEntry> offsetEntry : indexStartMap.entrySet()) {
+        IndexEntry entry = offsetEntry.getValue();
+        runningSize += entry.size;
+        if (runningSize >= MAX_ALLOCATION_SIZE) {
+          copyAndRemap(indexStartMap, offsetAccum, offsetEntry.getKey(), copyToOffset);
+          copyToOffset += runningSize - entry.size;
+          runningSize = entry.size;
+          offsetAccum.clear();
+        }
+        offsetAccum.add(offsetEntry.getKey());
+      }
+      if (offsetAccum.size() > 0) {
+        copyAndRemap(indexStartMap, offsetAccum, offsetAccum.get(0) + runningSize, copyToOffset);
+        copyToOffset += runningSize;
+      }
+      columnEntries.remove(indexKey);
+      RandomAccessFile randomFile = new RandomAccessFile(indexFile, "rw");
+      randomFile.setLength(copyToOffset);
+      updateIndexMap();

Review comment:
       How about metadata?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] kishoreg commented on pull request #5711: [WIP] - Supporting inverted index drop from v3 format

Posted by GitBox <gi...@apache.org>.
kishoreg commented on pull request #5711:
URL: https://github.com/apache/incubator-pinot/pull/5711#issuecomment-704705126


   Can we take this to completion?


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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org