You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2021/05/27 00:35:26 UTC

[incubator-pinot] branch master updated: feature/#6766 JSON and Startree index information in API (#6873)

This is an automated email from the ASF dual-hosted git repository.

jackie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new ab5ace8  feature/#6766 JSON and Startree index information in API (#6873)
ab5ace8 is described below

commit ab5ace882169f99c7d4f2b216bd680a4f08982dd
Author: Kriti Kathuria <38...@users.noreply.github.com>
AuthorDate: Thu May 27 06:04:58 2021 +0530

    feature/#6766 JSON and Startree index information in API (#6873)
    
    Expose info about JSON index and star-tree index
---
 .../api/resources/SegmentMetadataFetcher.java      | 79 ++++++++++++++++++----
 .../pinot/server/api/TablesResourceTest.java       |  1 +
 2 files changed, 67 insertions(+), 13 deletions(-)

diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java
index 01f98f3..fe24004 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java
@@ -20,34 +20,48 @@ package org.apache.pinot.server.api.resources;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager;
+import javax.annotation.Nullable;
 import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
 import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl;
-import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
 import org.apache.pinot.segment.spi.index.column.ColumnIndexContainer;
 import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl;
+import org.apache.pinot.segment.spi.index.startree.AggregationFunctionColumnPair;
+import org.apache.pinot.segment.spi.index.startree.StarTreeV2;
+import org.apache.pinot.segment.spi.index.startree.StarTreeV2Metadata;
 import org.apache.pinot.spi.utils.JsonUtils;
 
 /**
  * This is a wrapper class for fetching segment metadata related information.
  */
 public class SegmentMetadataFetcher {
+  private static final String COLUMN_INDEX_KEY = "indexes";
+  private static final String STAR_TREE_INDEX_KEY = "star-tree-index";
+
   private static final String BLOOM_FILTER = "bloom-filter";
   private static final String DICTIONARY = "dictionary";
   private static final String FORWARD_INDEX = "forward-index";
   private static final String INVERTED_INDEX = "inverted-index";
   private static final String NULL_VALUE_VECTOR_READER = "null-value-vector-reader";
   private static final String RANGE_INDEX = "range-index";
+  private static final String JSON_INDEX = "json-index";
 
   private static final String INDEX_NOT_AVAILABLE = "NO";
   private static final String INDEX_AVAILABLE = "YES";
 
+  private static final String STAR_TREE_DIMENSION_COLUMNS = "dimension-columns";
+  private static final String STAR_TREE_METRIC_AGGREGATIONS = "metric-aggregations";
+  private static final String STAR_TREE_MAX_LEAF_RECORDS = "max-leaf-records";
+  private static final String STAR_TREE_DIMENSION_COLUMNS_SKIPPED = "dimension-columns-skipped";
+  
+
   /**
    * This is a helper method that fetches the segment metadata for a given segment.
    * @param columns Columns to include for metadata
@@ -62,25 +76,22 @@ public class SegmentMetadataFetcher {
       columnSet = new HashSet<>(columns);
     }
     ObjectNode segmentMetadataJson = (ObjectNode) segmentMetadata.toJson(columnSet);
-    segmentMetadataJson.set("indexes", JsonUtils.objectToJsonNode(getIndexesForSegmentColumns(segmentDataManager)));
+    segmentMetadataJson.set(COLUMN_INDEX_KEY, JsonUtils.objectToJsonNode(getIndexesForSegmentColumns(segmentDataManager)));
+    segmentMetadataJson.set(STAR_TREE_INDEX_KEY, JsonUtils.objectToJsonNode((getStarTreeIndexesForSegment(segmentDataManager))));
     return JsonUtils.objectToString(segmentMetadataJson);
   }
 
   /**
    * Get the JSON object with the segment column's indexing metadata.
    */
+  @Nullable
   private static Map<String, Map<String, String>> getIndexesForSegmentColumns(SegmentDataManager segmentDataManager) {
-    Map<String, Map<String, String>> columnIndexMap = null;
-    if (segmentDataManager instanceof ImmutableSegmentDataManager) {
-      ImmutableSegmentDataManager immutableSegmentDataManager = (ImmutableSegmentDataManager) segmentDataManager;
-      ImmutableSegment immutableSegment = immutableSegmentDataManager.getSegment();
-      if (immutableSegment instanceof ImmutableSegmentImpl) {
-        ImmutableSegmentImpl immutableSegmentImpl = (ImmutableSegmentImpl) immutableSegment;
-        Map<String, ColumnIndexContainer> columnIndexContainerMap = immutableSegmentImpl.getIndexContainerMap();
-        columnIndexMap = getImmutableSegmentColumnIndexes(columnIndexContainerMap);
-      }
+    IndexSegment segment = segmentDataManager.getSegment();
+    if (segment instanceof ImmutableSegmentImpl) {
+      return getImmutableSegmentColumnIndexes(((ImmutableSegmentImpl) segment).getIndexContainerMap());
+    } else {
+      return null;
     }
-    return columnIndexMap;
   }
 
   /**
@@ -128,8 +139,50 @@ public class SegmentMetadataFetcher {
         indexStatus.put(RANGE_INDEX, INDEX_AVAILABLE);
       }
 
+      if (Objects.isNull(columnIndexContainer.getJsonIndex())){
+        indexStatus.put(JSON_INDEX, INDEX_NOT_AVAILABLE);
+      } else {
+        indexStatus.put(JSON_INDEX, INDEX_AVAILABLE);
+      }
+
       columnIndexMap.put(entry.getKey(), indexStatus);
     }
     return columnIndexMap;
   }
+
+  /**
+   * Get the JSON object containing star tree index details for a segment.
+   */
+  @Nullable
+  private static List<Map<String, Object>> getStarTreeIndexesForSegment(SegmentDataManager segmentDataManager) {
+    List<StarTreeV2> starTrees = segmentDataManager.getSegment().getStarTrees();
+    return starTrees != null ? getStarTreeIndexes(starTrees) : null;
+  }
+
+  /**
+   * Helper to loop over star trees of a segment to create a map containing star tree details.
+   */
+  private static List<Map<String, Object>> getStarTreeIndexes(List<StarTreeV2> starTrees){
+    List<Map<String, Object>> startreeDetails = new ArrayList<>();
+    for (StarTreeV2 starTree : starTrees) {
+      StarTreeV2Metadata starTreeMetadata = starTree.getMetadata();
+
+      Map<String, Object> starTreeIndexMap = new LinkedHashMap<>();
+
+      List<String> starTreeDimensions = starTreeMetadata.getDimensionsSplitOrder();
+      starTreeIndexMap.put(STAR_TREE_DIMENSION_COLUMNS, starTreeDimensions);
+
+      List<String> starTreeMetricAggregations = new ArrayList<>();
+      Set<AggregationFunctionColumnPair> functionColumnPairs = starTreeMetadata.getFunctionColumnPairs();
+      for (AggregationFunctionColumnPair functionColumnPair : functionColumnPairs) {
+        starTreeMetricAggregations.add(functionColumnPair.toColumnName());
+      }
+      starTreeIndexMap.put(STAR_TREE_METRIC_AGGREGATIONS, starTreeMetricAggregations);
+
+      starTreeIndexMap.put(STAR_TREE_MAX_LEAF_RECORDS, starTreeMetadata.getMaxLeafRecords());
+      starTreeIndexMap.put(STAR_TREE_DIMENSION_COLUMNS_SKIPPED, starTreeMetadata.getSkipStarNodeCreationForDimensions());
+      startreeDetails.add(starTreeIndexMap);
+    }
+    return startreeDetails;
+  }
 }
diff --git a/pinot-server/src/test/java/org/apache/pinot/server/api/TablesResourceTest.java b/pinot-server/src/test/java/org/apache/pinot/server/api/TablesResourceTest.java
index 59dfe64..29864ba 100644
--- a/pinot-server/src/test/java/org/apache/pinot/server/api/TablesResourceTest.java
+++ b/pinot-server/src/test/java/org/apache/pinot/server/api/TablesResourceTest.java
@@ -273,6 +273,7 @@ public class TablesResourceTest extends BaseResourceTest {
             .get(String.class));
     Assert.assertEquals(jsonResponse.get("columns").size(), 2);
     Assert.assertEquals(jsonResponse.get("indexes").size(), 17);
+    Assert.assertEquals(jsonResponse.get("star-tree-index").size(), 0);
 
     jsonResponse = JsonUtils.stringToJsonNode(
         (_webTarget.path(segmentMetadataPath).queryParam("columns", "*").request().get(String.class)));

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