kvOptions) {
+ String partitionSizeStr = kvOptions.get(PinotHintOptions.TableHintOptions.PARTITION_SIZE);
+ if (partitionSizeStr == null) {
+ return base;
+ }
+ try {
+ int explicitPartitionSize = Integer.parseInt(partitionSizeStr);
+ if (explicitPartitionSize == base.getPartitionSize()) {
+ return base;
+ }
+ LOGGER.debug("Override implicit table hint for {} with explicit partition size: {}", tableScan,
+ explicitPartitionSize);
+ return base.withPartitionSize(explicitPartitionSize);
+ } catch (NumberFormatException e) {
+ throw new IllegalArgumentException("Invalid partition size: " + partitionSizeStr + " for table: " + tableScan);
+ }
+ }
+
+ @Value.Immutable
+ public interface Config extends RelRule.Config {
+ @Nullable
+ WorkerManager getWorkerManager();
+
+ @Override
+ default PinotImplicitTableHintRule toRule() {
+ return new PinotImplicitTableHintRule(this);
+ }
+ }
+}
diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/TableOptions.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/TableOptions.java
new file mode 100644
index 000000000000..dfcbaa8ae3e1
--- /dev/null
+++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/TableOptions.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.calcite.rel.rules;
+
+import javax.annotation.Nullable;
+import org.immutables.value.Value;
+
+
+/**
+ * An internal interface used to generate the table options hint.
+ */
+@Value.Immutable
+public interface TableOptions {
+ String getPartitionKey();
+
+ String getPartitionFunction();
+
+ int getPartitionSize();
+
+ @Nullable
+ Integer getPartitionParallelism();
+}
diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java
index f14adb55a5b0..1681d41c9479 100644
--- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java
+++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java
@@ -22,6 +22,7 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.Properties;
import java.util.Set;
import javax.annotation.Nullable;
@@ -51,6 +52,7 @@
import org.apache.calcite.tools.FrameworkConfig;
import org.apache.calcite.tools.Frameworks;
import org.apache.calcite.tools.RelBuilder;
+import org.apache.pinot.calcite.rel.rules.PinotImplicitTableHintRule;
import org.apache.pinot.calcite.rel.rules.PinotQueryRuleSets;
import org.apache.pinot.calcite.rel.rules.PinotRelDistributionTraitRule;
import org.apache.pinot.calcite.rel.rules.PinotRuleUtils;
@@ -74,9 +76,11 @@
import org.apache.pinot.query.routing.WorkerManager;
import org.apache.pinot.query.type.TypeFactory;
import org.apache.pinot.query.validate.BytesCastVisitor;
+import org.apache.pinot.spi.utils.CommonConstants;
import org.apache.pinot.sql.parsers.CalciteSqlParser;
import org.apache.pinot.sql.parsers.SqlNodeAndOptions;
import org.apache.pinot.sql.parsers.parser.SqlPhysicalExplain;
+import org.immutables.value.Value;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -85,7 +89,14 @@
* The {@code QueryEnvironment} contains the main entrypoint for query planning.
*
* It provide the higher level entry interface to convert a SQL string into a {@link DispatchableSubPlan}.
+ * It is also used to execute some static analysis on the query like to determine if it can be compiled or get the
+ * tables involved in the query.
*/
+
+ //TODO: We should consider splitting this class in two: One that is used for parsing and one that is used for
+ // executing queries. This would allow us to remove the worker manager from the parsing environment and therefore
+ // make sure there is a worker manager when executing queries.
+@Value.Enclosing
public class QueryEnvironment {
private static final Logger LOGGER = LoggerFactory.getLogger(QueryEnvironment.class);
private static final CalciteConnectionConfig CONNECTION_CONFIG;
@@ -100,26 +111,64 @@ public class QueryEnvironment {
private final FrameworkConfig _config;
private final CalciteCatalogReader _catalogReader;
private final HepProgram _optProgram;
- private final HepProgram _traitProgram;
-
- // Pinot extensions
- private final TableCache _tableCache;
- private final WorkerManager _workerManager;
+ private final Config _envConfig;
- public QueryEnvironment(String database, TableCache tableCache, @Nullable WorkerManager workerManager) {
- PinotCatalog catalog = new PinotCatalog(tableCache, database);
+ public QueryEnvironment(Config config) {
+ _envConfig = config;
+ String database = config.getDatabase();
+ PinotCatalog catalog = new PinotCatalog(config.getTableCache(), database);
CalciteSchema rootSchema = CalciteSchema.createRootSchema(false, false, database, catalog);
_config = Frameworks.newConfigBuilder().traitDefs().operatorTable(PinotOperatorTable.instance())
.defaultSchema(rootSchema.plus()).sqlToRelConverterConfig(PinotRuleUtils.PINOT_SQL_TO_REL_CONFIG).build();
_catalogReader = new CalciteCatalogReader(rootSchema, List.of(database), _typeFactory, CONNECTION_CONFIG);
_optProgram = getOptProgram();
- _traitProgram = getTraitProgram();
- _tableCache = tableCache;
- _workerManager = workerManager;
}
- private PlannerContext getPlannerContext() {
- return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram);
+ public QueryEnvironment(String database, TableCache tableCache, @Nullable WorkerManager workerManager) {
+ this(configBuilder()
+ .database(database)
+ .tableCache(tableCache)
+ .workerManager(workerManager)
+ .build());
+ }
+
+ /**
+ * Returns a planner context that can be used to either parse, explain or execute a query.
+ */
+ private PlannerContext getPlannerContext(SqlNodeAndOptions sqlNodeAndOptions) {
+ WorkerManager workerManager = getWorkerManager(sqlNodeAndOptions);
+ HepProgram traitProgram = getTraitProgram(workerManager);
+ return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, traitProgram);
+ }
+
+ @Nullable
+ private WorkerManager getWorkerManager(SqlNodeAndOptions sqlNodeAndOptions) {
+ String inferPartitionHint = sqlNodeAndOptions.getOptions()
+ .get(CommonConstants.Broker.Request.QueryOptionKey.INFER_PARTITION_HINT);
+ WorkerManager workerManager = _envConfig.getWorkerManager();
+
+ if (inferPartitionHint == null) {
+ return _envConfig.defaultInferPartitionHint() ? workerManager : null;
+ }
+ switch (inferPartitionHint.toLowerCase()) {
+ case "true":
+ Objects.requireNonNull(workerManager, "WorkerManager is required in order to infer partition hint");
+ return workerManager;
+ case "false":
+ return null;
+ default:
+ throw new RuntimeException("Invalid value for query option '"
+ + CommonConstants.Broker.Request.QueryOptionKey.INFER_PARTITION_HINT + "': "
+ + inferPartitionHint);
+ }
+ }
+
+ /**
+ * Returns the planner context that should be used only for parsing queries.
+ */
+ private PlannerContext getParsingPlannerContext() {
+ HepProgram traitProgram = getTraitProgram(null);
+ return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, traitProgram);
}
/**
@@ -135,7 +184,7 @@ private PlannerContext getPlannerContext() {
* @return QueryPlannerResult containing the dispatchable query plan and the relRoot.
*/
public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId) {
- try (PlannerContext plannerContext = getPlannerContext()) {
+ try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions)) {
plannerContext.setOptions(sqlNodeAndOptions.getOptions());
RelRoot relRoot = compileQuery(sqlNodeAndOptions.getSqlNode(), plannerContext);
// TODO: current code only assume one SubPlan per query, but we should support multiple SubPlans per query.
@@ -150,6 +199,11 @@ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAn
}
}
+ @VisibleForTesting
+ public DispatchableSubPlan planQuery(String sqlQuery) {
+ return planQuery(sqlQuery, CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery), 0).getQueryPlan();
+ }
+
/**
* Explain a SQL query.
*
@@ -165,7 +219,7 @@ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAn
*/
public QueryPlannerResult explainQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId,
@Nullable AskingServerStageExplainer.OnServerExplainer onServerExplainer) {
- try (PlannerContext plannerContext = getPlannerContext()) {
+ try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions)) {
SqlExplain explain = (SqlExplain) sqlNodeAndOptions.getSqlNode();
plannerContext.setOptions(sqlNodeAndOptions.getOptions());
RelRoot relRoot = compileQuery(explain.getExplicandum(), plannerContext);
@@ -209,11 +263,6 @@ public QueryPlannerResult explainQuery(String sqlQuery, SqlNodeAndOptions sqlNod
}
}
- @VisibleForTesting
- public DispatchableSubPlan planQuery(String sqlQuery) {
- return planQuery(sqlQuery, CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery), 0).getQueryPlan();
- }
-
@VisibleForTesting
public String explainQuery(String sqlQuery, long requestId) {
SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery);
@@ -222,7 +271,7 @@ public String explainQuery(String sqlQuery, long requestId) {
}
public List getTableNamesForQuery(String sqlQuery) {
- try (PlannerContext plannerContext = getPlannerContext()) {
+ try (PlannerContext plannerContext = getParsingPlannerContext()) {
SqlNode sqlNode = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery).getSqlNode();
if (sqlNode.getKind().equals(SqlKind.EXPLAIN)) {
sqlNode = ((SqlExplain) sqlNode).getExplicandum();
@@ -239,7 +288,7 @@ public List getTableNamesForQuery(String sqlQuery) {
* Returns whether the query can be successfully compiled in this query environment
*/
public boolean canCompileQuery(String query) {
- try (PlannerContext plannerContext = getPlannerContext()) {
+ try (PlannerContext plannerContext = getParsingPlannerContext()) {
SqlNode sqlNode = CalciteSqlParser.compileToSqlNodeAndOptions(query).getSqlNode();
if (sqlNode.getKind().equals(SqlKind.EXPLAIN)) {
sqlNode = ((SqlExplain) sqlNode).getExplicandum();
@@ -353,7 +402,7 @@ private DispatchableSubPlan toDispatchableSubPlan(RelRoot relRoot, PlannerContex
@Nullable TransformationTracker.Builder tracker) {
SubPlan plan = PinotLogicalQueryPlanner.makePlan(relRoot, tracker);
PinotDispatchPlanner pinotDispatchPlanner =
- new PinotDispatchPlanner(plannerContext, _workerManager, requestId, _tableCache);
+ new PinotDispatchPlanner(plannerContext, _envConfig.getWorkerManager(), requestId, _envConfig.getTableCache());
return pinotDispatchPlanner.createDispatchableSubPlan(plan);
}
@@ -385,7 +434,7 @@ private static HepProgram getOptProgram() {
return hepProgramBuilder.build();
}
- private static HepProgram getTraitProgram() {
+ private static HepProgram getTraitProgram(@Nullable WorkerManager workerManager) {
HepProgramBuilder hepProgramBuilder = new HepProgramBuilder();
// Set the match order as BOTTOM_UP.
@@ -398,8 +447,44 @@ private static HepProgram getTraitProgram() {
}
// apply RelDistribution trait to all nodes
+ if (workerManager != null) {
+ hepProgramBuilder.addRuleInstance(PinotImplicitTableHintRule.withWorkerManager(workerManager));
+ }
hepProgramBuilder.addRuleInstance(PinotRelDistributionTraitRule.INSTANCE);
return hepProgramBuilder.build();
}
+
+ public static ImmutableQueryEnvironment.Config.Builder configBuilder() {
+ return ImmutableQueryEnvironment.Config.builder();
+ }
+
+ @Value.Immutable
+ public interface Config {
+ String getDatabase();
+
+ @Nullable // In theory nullable only in tests. We should fix LiteralOnlyBrokerRequestTest to not need this.
+ TableCache getTableCache();
+
+ /**
+ * Whether to apply partition hint by default or not.
+ *
+ * This is treated as the default value for the broker and it is expected to be obtained from a Pinot configuration.
+ * This default value can be always overridden at query level by the query option
+ * {@link CommonConstants.Broker.Request.QueryOptionKey#INFER_PARTITION_HINT}.
+ */
+ @Value.Default
+ default boolean defaultInferPartitionHint() {
+ return CommonConstants.Broker.DEFAULT_INFER_PARTITION_HINT;
+ }
+
+ /**
+ * Returns the worker manager.
+ *
+ * This is used whenever the query needs to be executed, but can be null when the QueryEnvironment will be used
+ * just to execute some static analysis on the query like parsing it or getting the tables involved in the query.
+ */
+ @Nullable
+ WorkerManager getWorkerManager();
+ }
}
diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/ExplainNodeSimplifier.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/ExplainNodeSimplifier.java
index e7a1b8d48d03..17b0b02fe1ed 100644
--- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/ExplainNodeSimplifier.java
+++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/ExplainNodeSimplifier.java
@@ -21,10 +21,7 @@
import com.google.common.base.CaseFormat;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
-import org.apache.pinot.common.proto.Plan;
import org.apache.pinot.core.query.reduce.ExplainPlanDataTableReducer;
import org.apache.pinot.query.planner.plannode.AggregateNode;
import org.apache.pinot.query.planner.plannode.ExchangeNode;
@@ -54,10 +51,8 @@
* Its title must contain the text {@code Combine}
*
*
- * Also nodes with only one input are not simplifiable by definition.
- *
- * Simplified nodes will have a new attribute {@code repeated} that will contain the number of times the node was
- * repeated.
+ * The simplification process merges the inputs of the node into a single node.
+ * As a corollary, nodes with only one input are already simplified by definition.
*/
public class ExplainNodeSimplifier {
private static final Logger LOGGER = LoggerFactory.getLogger(ExplainNodeSimplifier.class);
@@ -73,8 +68,6 @@ public static PlanNode simplifyNode(PlanNode root) {
}
private static class Visitor implements PlanNodeVisitor {
- private static final String REPEAT_ATTRIBUTE_KEY = "repeated";
-
private PlanNode defaultNode(PlanNode node) {
List inputs = node.getInputs();
List newInputs = simplifyChildren(inputs);
@@ -159,13 +152,8 @@ public PlanNode visitExplained(ExplainedNode node, Void context) {
}
child1 = merged;
}
- Map attributes = new HashMap<>(node.getAttributes());
- Plan.ExplainNode.AttributeValue repeatedValue = Plan.ExplainNode.AttributeValue.newBuilder()
- .setLong(simplifiedChildren.size())
- .build();
- attributes.put(REPEAT_ATTRIBUTE_KEY, repeatedValue);
return new ExplainedNode(node.getStageId(), node.getDataSchema(), node.getNodeHint(),
- Collections.singletonList(child1), node.getTitle(), attributes);
+ Collections.singletonList(child1), node.getTitle(), node.getAttributes());
}
private List simplifyChildren(List children) {
diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java
index 1541a8b6742a..aa2e44173b4e 100644
--- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java
+++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java
@@ -18,14 +18,18 @@
*/
package org.apache.pinot.query.planner.explain;
+import com.google.common.base.CaseFormat;
import com.google.common.collect.Streams;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Set;
import javax.annotation.Nullable;
import org.apache.pinot.common.proto.Plan;
import org.apache.pinot.core.operator.ExplainAttributeBuilder;
+import org.apache.pinot.core.query.reduce.ExplainPlanDataTableReducer;
import org.apache.pinot.query.planner.plannode.AggregateNode;
import org.apache.pinot.query.planner.plannode.ExchangeNode;
import org.apache.pinot.query.planner.plannode.ExplainedNode;
@@ -93,6 +97,8 @@ public static PlanNode mergePlans(PlanNode plan1, PlanNode plan2, boolean verbos
}
private static class Visitor implements PlanNodeVisitor {
+ public static final String COMBINE
+ = CaseFormat.UPPER_UNDERSCORE.to(CaseFormat.UPPER_CAMEL, ExplainPlanDataTableReducer.COMBINE);
private final boolean _verbose;
public Visitor(boolean verbose) {
@@ -441,7 +447,12 @@ public PlanNode visitExplained(ExplainedNode node, PlanNode context) {
Map selfAttributes = node.getAttributes();
Map otherAttributes = otherNode.getAttributes();
- List children = mergeChildren(node, context);
+ List children;
+ if (node.getTitle().contains(COMBINE)) {
+ children = mergeCombineChildren(node, otherNode);
+ } else {
+ children = mergeChildren(node, context);
+ }
if (children == null) {
return null;
}
@@ -486,23 +497,23 @@ public PlanNode visitExplained(ExplainedNode node, PlanNode context) {
if (selfValue.hasLong() && otherValue.hasLong()) { // If both are long, add them
attributeBuilder.putLong(selfEntry.getKey(), selfValue.getLong() + otherValue.getLong());
} else { // Otherwise behave as if they are idempotent
- if (!Objects.equals(otherValue, selfEntry.getValue())) {
+ if (!Objects.equals(otherValue, selfValue)) {
return null;
}
- attributeBuilder.putAttribute(selfEntry.getKey(), selfEntry.getValue());
+ attributeBuilder.putAttribute(selfEntry.getKey(), selfValue);
}
break;
}
case IDEMPOTENT: {
- if (!Objects.equals(otherValue, selfEntry.getValue())) {
+ if (!Objects.equals(otherValue, selfValue)) {
return null;
}
- attributeBuilder.putAttribute(selfEntry.getKey(), selfEntry.getValue());
+ attributeBuilder.putAttribute(selfEntry.getKey(), selfValue);
break;
}
case IGNORABLE: {
- if (Objects.equals(otherValue, selfEntry.getValue())) {
- attributeBuilder.putAttribute(selfEntry.getKey(), selfEntry.getValue());
+ if (Objects.equals(otherValue, selfValue)) {
+ attributeBuilder.putAttribute(selfEntry.getKey(), selfValue);
} else if (_verbose) {
// If mode is verbose, we will not merge the nodes when an ignorable attribute is different
return null;
@@ -518,13 +529,49 @@ public PlanNode visitExplained(ExplainedNode node, PlanNode context) {
}
for (Map.Entry otherEntry : otherAttributes.entrySet()) {
Plan.ExplainNode.AttributeValue selfValue = selfAttributes.get(otherEntry.getKey());
- if (selfValue == null) { // otherwise it has already been merged
- attributeBuilder.putAttribute(otherEntry.getKey(), otherEntry.getValue());
+ if (selfValue != null) { // it has already been merged
+ continue;
+ }
+ switch (otherEntry.getValue().getMergeType()) {
+ case DEFAULT:
+ attributeBuilder.putAttribute(otherEntry.getKey(), otherEntry.getValue());
+ break;
+ case IGNORABLE:
+ if (_verbose) {
+ return null;
+ }
+ break;
+ case IDEMPOTENT:
+ case UNRECOGNIZED:
+ default:
+ return null;
}
}
return new ExplainedNode(node.getStageId(), node.getDataSchema(), node.getNodeHint(), children, node.getTitle(),
attributeBuilder.build());
}
}
+
+ private List mergeCombineChildren(ExplainedNode node1, ExplainedNode node2) {
+ List mergedChildren = new ArrayList<>(node1.getInputs().size() + node2.getInputs().size());
+
+ Set pendingOn2 = new HashSet<>(node2.getInputs());
+ for (PlanNode input1 : node1.getInputs()) {
+ PlanNode merged = null;
+ for (PlanNode input2 : pendingOn2) {
+ merged = mergePlans(input1, input2);
+ if (merged != null) {
+ pendingOn2.remove(input2);
+ break;
+ }
+ }
+ mergedChildren.add(merged != null ? merged : input1);
+ }
+ mergedChildren.addAll(pendingOn2);
+
+ mergedChildren.sort(PlanNodeSorter.DefaultComparator.INSTANCE);
+
+ return mergedChildren;
+ }
}
}
diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java
index ba3a709fed00..6af598bb3da7 100644
--- a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java
+++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java
@@ -33,6 +33,8 @@
import org.apache.calcite.rel.RelDistribution;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pinot.calcite.rel.hint.PinotHintOptions;
+import org.apache.pinot.calcite.rel.rules.ImmutableTableOptions;
+import org.apache.pinot.calcite.rel.rules.TableOptions;
import org.apache.pinot.core.routing.RoutingManager;
import org.apache.pinot.core.routing.RoutingTable;
import org.apache.pinot.core.routing.TablePartitionInfo;
@@ -407,8 +409,10 @@ private void assignWorkersToPartitionedLeafFragment(DispatchablePlanMetadata met
PinotHintOptions.TableHintOptions.PARTITION_PARALLELISM, partitionParallelism);
String tableName = metadata.getScannedTables().get(0);
- PartitionTableInfo partitionTableInfo =
- getPartitionTableInfo(tableName, partitionKey, numPartitions, partitionFunction);
+ // calculates the partition table info using the routing manager
+ PartitionTableInfo partitionTableInfo = calculatePartitionTableInfo(tableName);
+ // verifies that the partition table obtained from routing manager is compatible with the hint options
+ checkPartitionInfoMap(partitionTableInfo, tableName, partitionKey, numPartitions, partitionFunction);
// Pick one server per partition
// NOTE: Pick server based on the request id so that the same server is picked across different table scan when the
@@ -440,8 +444,21 @@ private void assignWorkersToPartitionedLeafFragment(DispatchablePlanMetadata met
metadata.setPartitionParallelism(partitionParallelism);
}
- private PartitionTableInfo getPartitionTableInfo(String tableName, String partitionKey, int numPartitions,
- String partitionFunction) {
+ @Nullable
+ public TableOptions inferTableOptions(String tableName) {
+ try {
+ PartitionTableInfo partitionTableInfo = calculatePartitionTableInfo(tableName);
+ return ImmutableTableOptions.builder()
+ .partitionFunction(partitionTableInfo._partitionFunction)
+ .partitionKey(partitionTableInfo._partitionKey)
+ .partitionSize(partitionTableInfo._numPartitions)
+ .build();
+ } catch (IllegalStateException e) {
+ return null;
+ }
+ }
+
+ private PartitionTableInfo calculatePartitionTableInfo(String tableName) {
TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
if (tableType == null) {
String offlineTableName = TableNameBuilder.OFFLINE.tableNameWithType(tableName);
@@ -450,19 +467,27 @@ private PartitionTableInfo getPartitionTableInfo(String tableName, String partit
boolean realtimeRoutingExists = _routingManager.routingExists(realtimeTableName);
Preconditions.checkState(offlineRoutingExists || realtimeRoutingExists, "Routing doesn't exist for table: %s",
tableName);
+
if (offlineRoutingExists && realtimeRoutingExists) {
+ TablePartitionInfo offlineTpi = _routingManager.getTablePartitionInfo(offlineTableName);
+ Preconditions.checkState(offlineTpi != null, "Failed to find table partition info for table: %s",
+ offlineTableName);
+ TablePartitionInfo realtimeTpi = _routingManager.getTablePartitionInfo(realtimeTableName);
+ Preconditions.checkState(realtimeTpi != null, "Failed to find table partition info for table: %s",
+ realtimeTableName);
// For hybrid table, find the common servers for each partition
TimeBoundaryInfo timeBoundaryInfo = _routingManager.getTimeBoundaryInfo(offlineTableName);
// Ignore OFFLINE side when time boundary info is unavailable
if (timeBoundaryInfo == null) {
- return getRealtimePartitionTableInfo(realtimeTableName, partitionKey, numPartitions, partitionFunction);
+ return PartitionTableInfo.fromTablePartitionInfo(realtimeTpi, TableType.REALTIME);
}
- TablePartitionInfo.PartitionInfo[] offlinePartitionInfoMap =
- getTablePartitionInfo(offlineTableName, partitionKey, numPartitions,
- partitionFunction).getPartitionInfoMap();
- TablePartitionInfo.PartitionInfo[] realtimePartitionInfoMap =
- getTablePartitionInfo(realtimeTableName, partitionKey, numPartitions,
- partitionFunction).getPartitionInfoMap();
+
+ verifyCompatibility(offlineTpi, realtimeTpi);
+
+ TablePartitionInfo.PartitionInfo[] offlinePartitionInfoMap = offlineTpi.getPartitionInfoMap();
+ TablePartitionInfo.PartitionInfo[] realtimePartitionInfoMap = realtimeTpi.getPartitionInfoMap();
+
+ int numPartitions = offlineTpi.getNumPartitions();
PartitionInfo[] partitionInfoMap = new PartitionInfo[numPartitions];
for (int i = 0; i < numPartitions; i++) {
TablePartitionInfo.PartitionInfo offlinePartitionInfo = offlinePartitionInfoMap[i];
@@ -484,79 +509,117 @@ private PartitionTableInfo getPartitionTableInfo(String tableName, String partit
fullyReplicatedServers.retainAll(realtimePartitionInfo._fullyReplicatedServers);
Preconditions.checkState(!fullyReplicatedServers.isEmpty(),
"Failed to find fully replicated server for partition: %s in hybrid table: %s", i, tableName);
- partitionInfoMap[i] = new PartitionInfo(fullyReplicatedServers, offlinePartitionInfo._segments,
- realtimePartitionInfo._segments);
+ partitionInfoMap[i] = new PartitionInfo(
+ fullyReplicatedServers, offlinePartitionInfo._segments, realtimePartitionInfo._segments);
}
- return new PartitionTableInfo(partitionInfoMap, timeBoundaryInfo);
+ return new PartitionTableInfo(partitionInfoMap, timeBoundaryInfo, offlineTpi.getPartitionColumn(),
+ numPartitions, offlineTpi.getPartitionFunctionName());
} else if (offlineRoutingExists) {
- return getOfflinePartitionTableInfo(offlineTableName, partitionKey, numPartitions, partitionFunction);
+ return getOfflinePartitionTableInfo(offlineTableName);
} else {
- return getRealtimePartitionTableInfo(realtimeTableName, partitionKey, numPartitions, partitionFunction);
+ return getRealtimePartitionTableInfo(realtimeTableName);
}
} else {
if (tableType == TableType.OFFLINE) {
- return getOfflinePartitionTableInfo(tableName, partitionKey, numPartitions, partitionFunction);
+ return getOfflinePartitionTableInfo(tableName);
} else {
- return getRealtimePartitionTableInfo(tableName, partitionKey, numPartitions, partitionFunction);
+ return getRealtimePartitionTableInfo(tableName);
}
}
}
- private TablePartitionInfo getTablePartitionInfo(String tableNameWithType, String partitionKey, int numPartitions,
- String partitionFunction) {
- TablePartitionInfo tablePartitionInfo = _routingManager.getTablePartitionInfo(tableNameWithType);
- Preconditions.checkState(tablePartitionInfo != null, "Failed to find table partition info for table: %s",
- tableNameWithType);
- Preconditions.checkState(tablePartitionInfo.getPartitionColumn().equals(partitionKey),
+ private static void verifyCompatibility(TablePartitionInfo offlineTpi, TablePartitionInfo realtimeTpi)
+ throws IllegalArgumentException {
+ Preconditions.checkState(offlineTpi.getPartitionColumn().equals(realtimeTpi.getPartitionColumn()),
+ "Partition column mismatch for hybrid table %s: %s offline vs %s online",
+ offlineTpi.getTableNameWithType(), offlineTpi.getPartitionColumn(), realtimeTpi.getPartitionColumn());
+ Preconditions.checkState(offlineTpi.getNumPartitions() == realtimeTpi.getNumPartitions(),
+ "Partition size mismatch for hybrid table %s: %s offline vs %s online",
+ offlineTpi.getTableNameWithType(), offlineTpi.getNumPartitions(), realtimeTpi.getNumPartitions());
+ Preconditions.checkState(
+ offlineTpi.getPartitionFunctionName().equalsIgnoreCase(realtimeTpi.getPartitionFunctionName()),
+ "Partition function mismatch for hybrid table %s: %s offline vs %s online",
+ offlineTpi.getTableNameWithType(), offlineTpi.getPartitionFunctionName(),
+ realtimeTpi.getPartitionFunctionName());
+ }
+
+ /**
+ * Verifies that the partition info maps from the table partition info are compatible with the information supplied
+ * as arguments.
+ */
+ private void checkPartitionInfoMap(PartitionTableInfo partitionTableInfo, String tableNameWithType,
+ String partitionKey, int numPartitions, String partitionFunction) {
+ Preconditions.checkState(partitionTableInfo._partitionKey.equals(partitionKey),
"Partition key: %s does not match partition column: %s for table: %s", partitionKey,
- tablePartitionInfo.getPartitionColumn(), tableNameWithType);
- Preconditions.checkState(tablePartitionInfo.getNumPartitions() == numPartitions,
+ partitionTableInfo._partitionKey, tableNameWithType);
+ Preconditions.checkState(partitionTableInfo._numPartitions == numPartitions,
"Partition size mismatch (hint: %s, table: %s) for table: %s", numPartitions,
- tablePartitionInfo.getNumPartitions(), tableNameWithType);
- Preconditions.checkState(tablePartitionInfo.getPartitionFunctionName().equalsIgnoreCase(partitionFunction),
+ partitionTableInfo._numPartitions, tableNameWithType);
+ Preconditions.checkState(partitionTableInfo._partitionFunction.equalsIgnoreCase(partitionFunction),
"Partition function mismatch (hint: %s, table: %s) for table %s", partitionFunction,
- tablePartitionInfo.getPartitionFunctionName(), tableNameWithType);
- Preconditions.checkState(tablePartitionInfo.getSegmentsWithInvalidPartition().isEmpty(),
- "Find %s segments with invalid partition for table: %s",
- tablePartitionInfo.getSegmentsWithInvalidPartition().size(), tableNameWithType);
- return tablePartitionInfo;
+ partitionTableInfo._partitionFunction, tableNameWithType);
}
- private PartitionTableInfo getOfflinePartitionTableInfo(String offlineTableName, String partitionKey,
- int numPartitions, String partitionFunction) {
- TablePartitionInfo.PartitionInfo[] tablePartitionInfoArr =
- getTablePartitionInfo(offlineTableName, partitionKey, numPartitions, partitionFunction).getPartitionInfoMap();
- PartitionInfo[] partitionInfoMap = new PartitionInfo[numPartitions];
- for (int i = 0; i < numPartitions; i++) {
- TablePartitionInfo.PartitionInfo partitionInfo = tablePartitionInfoArr[i];
- if (partitionInfo != null) {
- partitionInfoMap[i] = new PartitionInfo(partitionInfo._fullyReplicatedServers, partitionInfo._segments, null);
- }
- }
- return new PartitionTableInfo(partitionInfoMap, null);
+ private PartitionTableInfo getOfflinePartitionTableInfo(String offlineTableName) {
+ TablePartitionInfo offlineTpi = _routingManager.getTablePartitionInfo(offlineTableName);
+ Preconditions.checkState(offlineTpi != null, "Failed to find table partition info for table: %s",
+ offlineTableName);
+ return PartitionTableInfo.fromTablePartitionInfo(offlineTpi, TableType.OFFLINE);
}
- private PartitionTableInfo getRealtimePartitionTableInfo(String realtimeTableName, String partitionKey,
- int numPartitions, String partitionFunction) {
- TablePartitionInfo.PartitionInfo[] tablePartitionInfoArr =
- getTablePartitionInfo(realtimeTableName, partitionKey, numPartitions, partitionFunction).getPartitionInfoMap();
- PartitionInfo[] partitionInfoMap = new PartitionInfo[numPartitions];
- for (int i = 0; i < numPartitions; i++) {
- TablePartitionInfo.PartitionInfo partitionInfo = tablePartitionInfoArr[i];
- if (partitionInfo != null) {
- partitionInfoMap[i] = new PartitionInfo(partitionInfo._fullyReplicatedServers, null, partitionInfo._segments);
- }
- }
- return new PartitionTableInfo(partitionInfoMap, null);
+ private PartitionTableInfo getRealtimePartitionTableInfo(String realtimeTableName) {
+ TablePartitionInfo realtimeTpi = _routingManager.getTablePartitionInfo(realtimeTableName);
+ Preconditions.checkState(realtimeTpi != null, "Failed to find table partition info for table: %s",
+ realtimeTableName);
+ return PartitionTableInfo.fromTablePartitionInfo(realtimeTpi, TableType.REALTIME);
}
private static class PartitionTableInfo {
final PartitionInfo[] _partitionInfoMap;
+ @Nullable
final TimeBoundaryInfo _timeBoundaryInfo;
+ final String _partitionKey;
+ final int _numPartitions;
+ final String _partitionFunction;
- PartitionTableInfo(PartitionInfo[] partitionInfoMap, @Nullable TimeBoundaryInfo timeBoundaryInfo) {
+ PartitionTableInfo(PartitionInfo[] partitionInfoMap, @Nullable TimeBoundaryInfo timeBoundaryInfo,
+ String partitionKey, int numPartitions, String partitionFunction) {
_partitionInfoMap = partitionInfoMap;
_timeBoundaryInfo = timeBoundaryInfo;
+ _partitionKey = partitionKey;
+ _numPartitions = numPartitions;
+ _partitionFunction = partitionFunction;
+ }
+
+ public static PartitionTableInfo fromTablePartitionInfo(
+ TablePartitionInfo tablePartitionInfo, TableType tableType) {
+ if (!tablePartitionInfo.getSegmentsWithInvalidPartition().isEmpty()) {
+ throw new IllegalStateException("Find " + tablePartitionInfo.getSegmentsWithInvalidPartition().size()
+ + " segments with invalid partition");
+ }
+
+ int numPartitions = tablePartitionInfo.getNumPartitions();
+ TablePartitionInfo.PartitionInfo[] tablePartitionInfoMap = tablePartitionInfo.getPartitionInfoMap();
+ PartitionInfo[] workerPartitionInfoMap = new PartitionInfo[numPartitions];
+ for (int i = 0; i < numPartitions; i++) {
+ TablePartitionInfo.PartitionInfo partitionInfo = tablePartitionInfoMap[i];
+ if (partitionInfo != null) {
+ switch (tableType) {
+ case OFFLINE:
+ workerPartitionInfoMap[i] =
+ new PartitionInfo(partitionInfo._fullyReplicatedServers, partitionInfo._segments, null);
+ break;
+ case REALTIME:
+ workerPartitionInfoMap[i] =
+ new PartitionInfo(partitionInfo._fullyReplicatedServers, null, partitionInfo._segments);
+ break;
+ default:
+ throw new IllegalStateException("Unsupported table type: " + tableType);
+ }
+ }
+ }
+ return new PartitionTableInfo(workerPartitionInfoMap, null, tablePartitionInfo.getPartitionColumn(),
+ numPartitions, tablePartitionInfo.getPartitionFunctionName());
}
}
diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java
index 5b80d3506545..a0739162a615 100644
--- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java
+++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java
@@ -253,7 +253,8 @@ public void processTimeSeriesQuery(String serializedPlan, Map me
final Consumer handleErrors = (t) -> {
Map errorMetadata = new HashMap<>();
errorMetadata.put(WorkerResponseMetadataKeys.ERROR_TYPE, t.getClass().getSimpleName());
- errorMetadata.put(WorkerResponseMetadataKeys.ERROR_MESSAGE, t.getMessage());
+ errorMetadata.put(WorkerResponseMetadataKeys.ERROR_MESSAGE, t.getMessage() == null
+ ? "Unknown error: no message" : t.getMessage());
responseObserver.onNext(Worker.TimeSeriesResponse.newBuilder().putAllMetadata(errorMetadata).build());
responseObserver.onCompleted();
};
@@ -280,6 +281,7 @@ public void processTimeSeriesQuery(String serializedPlan, Map me
}
});
} catch (Throwable t) {
+ LOGGER.error("Error running time-series query", t);
handleErrors.accept(t);
}
}
diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java
index 1f3f3a20fc32..41501f69383e 100644
--- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java
+++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java
@@ -58,6 +58,7 @@ public class MultistageGroupByExecutor {
private final AggType _aggType;
private final DataSchema _resultSchema;
private final int _numGroupsLimit;
+ private final boolean _filteredAggregationsSkipEmptyGroups;
// Group By Result holders for each mode
private final GroupByResultHolder[] _aggregateResultHolders;
@@ -79,6 +80,10 @@ public MultistageGroupByExecutor(int[] groupKeyIds, AggregationFunction[] aggFun
int maxInitialResultHolderCapacity = getMaxInitialResultHolderCapacity(opChainMetadata, nodeHint);
_numGroupsLimit = getNumGroupsLimit(opChainMetadata, nodeHint);
+ // By default, we compute all groups for SQL compliant results. However, we allow overriding this behavior via
+ // query option for improved performance.
+ _filteredAggregationsSkipEmptyGroups = QueryOptionsUtils.isFilteredAggregationsSkipEmptyGroups(opChainMetadata);
+
int numFunctions = aggFunctions.length;
if (!aggType.isInputIntermediateFormat()) {
_aggregateResultHolders = new GroupByResultHolder[numFunctions];
@@ -241,6 +246,12 @@ private void processAggregate(TransferableBlock block) {
aggFunction.aggregateGroupBySV(numMatchedRows, filteredIntKeys, groupByResultHolder, blockValSetMap);
}
}
+ if (intKeys == null && !_filteredAggregationsSkipEmptyGroups) {
+ // _groupIdGenerator should still have all the groups even if there are only filtered aggregates for SQL
+ // compliant results. However, if the query option to skip empty groups is set, we avoid this step for
+ // improved performance.
+ generateGroupByKeys(block);
+ }
}
}
diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java
index e6d6d4342481..8fb1a6c75852 100644
--- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java
+++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java
@@ -113,8 +113,8 @@ protected QueryEnvironment.QueryPlannerResult planQuery(String sql) {
protected QueryDispatcher.QueryResult queryRunner(String sql, boolean trace) {
long requestId = REQUEST_ID_GEN.getAndIncrement();
SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sql);
- QueryEnvironment.QueryPlannerResult queryPlannerResult =
- _queryEnvironment.planQuery(sql, sqlNodeAndOptions, requestId);
+ QueryEnvironment.QueryPlannerResult queryPlannerResult = _queryEnvironment.planQuery(sql, sqlNodeAndOptions,
+ requestId);
DispatchableSubPlan dispatchableSubPlan = queryPlannerResult.getQueryPlan();
Map requestMetadataMap = new HashMap<>();
requestMetadataMap.put(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID, String.valueOf(requestId));
diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/timeseries/PhysicalTimeSeriesPlanVisitorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/timeseries/PhysicalTimeSeriesPlanVisitorTest.java
index 929e2669cf6a..8f877aec016d 100644
--- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/timeseries/PhysicalTimeSeriesPlanVisitorTest.java
+++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/timeseries/PhysicalTimeSeriesPlanVisitorTest.java
@@ -54,7 +54,7 @@ public void testCompileQueryContext() {
assertEquals(queryContext.getTimeSeriesContext().getTimeColumn(), timeColumn);
assertEquals(queryContext.getTimeSeriesContext().getValueExpression().getIdentifier(), "orderCount");
assertEquals(queryContext.getFilter().toString(),
- "(cityName = 'Chicago' AND orderTime >= '1000' AND orderTime <= '2000')");
+ "(cityName = 'Chicago' AND orderTime >= '1000' AND orderTime < '2000')");
}
// Case-2: With offset, complex group-by expression, complex value, and non-empty filter
{
@@ -75,7 +75,7 @@ public void testCompileQueryContext() {
assertEquals(queryContext.getTimeSeriesContext().getValueExpression().toString(), "times(orderCount,'2')");
assertNotNull(queryContext.getFilter());
assertEquals(queryContext.getFilter().toString(),
- "(cityName = 'Chicago' AND orderTime >= '990' AND orderTime <= '1990')");
+ "(cityName = 'Chicago' AND orderTime >= '990' AND orderTime < '1990')");
}
}
}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriter.java
index fadcce827e0a..0a6bd4792569 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriter.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriter.java
@@ -23,6 +23,7 @@
import java.io.IOException;
import java.math.BigDecimal;
import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.pinot.segment.local.utils.ArraySerDeUtils;
import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
import org.apache.pinot.spi.utils.BigDecimalUtils;
@@ -97,54 +98,34 @@ public void putBytes(byte[] value) {
writeChunkIfNecessary();
}
- // Note: some duplication is tolerated between these overloads for the sake of memory efficiency
+ @Override
+ public void putIntMV(int[] values) {
+ putBytes(ArraySerDeUtils.serializeIntArrayWithLength(values));
+ }
+
+ @Override
+ public void putLongMV(long[] values) {
+ putBytes(ArraySerDeUtils.serializeLongArrayWithLength(values));
+ }
+
+ @Override
+ public void putFloatMV(float[] values) {
+ putBytes(ArraySerDeUtils.serializeFloatArrayWithLength(values));
+ }
+
+ @Override
+ public void putDoubleMV(double[] values) {
+ putBytes(ArraySerDeUtils.serializeDoubleArrayWithLength(values));
+ }
@Override
public void putStringMV(String[] values) {
- // the entire String[] will be encoded as a single string, write the header here
- _chunkBuffer.putInt(_chunkHeaderOffset, _chunkDataOffSet);
- _chunkHeaderOffset += CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE;
- // write all the strings into the data buffer as if it's a single string,
- // but with its own embedded header so offsets to strings within the body
- // can be located
- _chunkBuffer.putInt(_chunkDataOffSet, values.length);
- _chunkDataOffSet += Integer.BYTES;
- int headerSize = Integer.BYTES * values.length;
- int bodyPosition = _chunkDataOffSet + headerSize;
- _chunkBuffer.position(bodyPosition);
- int bodySize = 0;
- for (int i = 0, h = _chunkDataOffSet; i < values.length; i++, h += Integer.BYTES) {
- byte[] utf8 = values[i].getBytes(UTF_8);
- _chunkBuffer.putInt(h, utf8.length);
- _chunkBuffer.put(utf8);
- bodySize += utf8.length;
- }
- _chunkDataOffSet += headerSize + bodySize;
- writeChunkIfNecessary();
+ putBytes(ArraySerDeUtils.serializeStringArray(values));
}
@Override
public void putBytesMV(byte[][] values) {
- // the entire byte[][] will be encoded as a single string, write the header here
- _chunkBuffer.putInt(_chunkHeaderOffset, _chunkDataOffSet);
- _chunkHeaderOffset += CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE;
- // write all the byte[]s into the data buffer as if it's a single byte[],
- // but with its own embedded header so offsets to byte[]s within the body
- // can be located
- _chunkBuffer.putInt(_chunkDataOffSet, values.length);
- _chunkDataOffSet += Integer.BYTES;
- int headerSize = Integer.BYTES * values.length;
- int bodyPosition = _chunkDataOffSet + headerSize;
- _chunkBuffer.position(bodyPosition);
- int bodySize = 0;
- for (int i = 0, h = _chunkDataOffSet; i < values.length; i++, h += Integer.BYTES) {
- byte[] bytes = values[i];
- _chunkBuffer.putInt(h, bytes.length);
- _chunkBuffer.put(bytes);
- bodySize += bytes.length;
- }
- _chunkDataOffSet += headerSize + bodySize;
- writeChunkIfNecessary();
+ putBytes(ArraySerDeUtils.serializeBytesArray(values));
}
private void writeChunkIfNecessary() {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV4.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV4.java
index 440808a6b0bd..332c52d0c59f 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV4.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV4.java
@@ -30,6 +30,7 @@
import javax.annotation.concurrent.NotThreadSafe;
import org.apache.commons.io.FileUtils;
import org.apache.pinot.segment.local.io.compression.ChunkCompressorFactory;
+import org.apache.pinot.segment.local.utils.ArraySerDeUtils;
import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
import org.apache.pinot.segment.spi.compression.ChunkCompressor;
import org.apache.pinot.segment.spi.memory.CleanerUtil;
@@ -37,8 +38,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
/**
* Chunk-based raw (non-dictionary-encoded) forward index writer where each chunk contains variable number of docs, and
@@ -77,7 +76,9 @@
public class VarByteChunkForwardIndexWriterV4 implements VarByteChunkWriter {
public static final int VERSION = 4;
- private static final Logger LOGGER = LoggerFactory.getLogger(VarByteChunkForwardIndexWriterV4.class);
+ // Use the run-time concrete class to retrieve the logger
+ protected final Logger _logger = LoggerFactory.getLogger(getClass());
+
private static final String DATA_BUFFER_SUFFIX = ".buf";
private final File _dataBuffer;
@@ -106,11 +107,16 @@ public VarByteChunkForwardIndexWriterV4(File file, ChunkCompressionType compress
writeHeader(_chunkCompressor.compressionType(), chunkSize);
}
+ // Child class must override this class instance method
+ protected int getVersion() {
+ return VERSION;
+ }
+
private void writeHeader(ChunkCompressionType compressionType, int targetDecompressedChunkSize)
throws IOException {
// keep metadata BE for backwards compatibility
// (e.g. the version needs to be read by a factory which assumes BE)
- _output.writeInt(VERSION);
+ _output.writeInt(getVersion());
_output.writeInt(targetDecompressedChunkSize);
_output.writeInt(compressionType.getValue());
// reserve a slot to write the data offset into
@@ -145,49 +151,33 @@ public void putBytes(byte[] bytes) {
}
@Override
- public void putStringMV(String[] values) {
- // num values + length of each value
- int headerSize = Integer.BYTES + Integer.BYTES * values.length;
- int size = headerSize;
- byte[][] stringBytes = new byte[values.length][];
- for (int i = 0; i < values.length; i++) {
- stringBytes[i] = values[i].getBytes(UTF_8);
- size += stringBytes[i].length;
- }
+ public void putIntMV(int[] values) {
+ putBytes(ArraySerDeUtils.serializeIntArrayWithLength(values));
+ }
- // Format : [numValues][length1][length2]...[lengthN][value1][value2]...[valueN]
- byte[] serializedBytes = new byte[size];
- ByteBuffer byteBuffer = ByteBuffer.wrap(serializedBytes);
- byteBuffer.putInt(values.length);
- byteBuffer.position(headerSize);
- for (int i = 0; i < values.length; i++) {
- byteBuffer.putInt((i + 1) * Integer.BYTES, stringBytes[i].length);
- byteBuffer.put(stringBytes[i]);
- }
+ @Override
+ public void putLongMV(long[] values) {
+ putBytes(ArraySerDeUtils.serializeLongArrayWithLength(values));
+ }
- putBytes(serializedBytes);
+ @Override
+ public void putFloatMV(float[] values) {
+ putBytes(ArraySerDeUtils.serializeFloatArrayWithLength(values));
}
@Override
- public void putBytesMV(byte[][] values) {
- // num values + length of each value
- int headerSize = Integer.BYTES + Integer.BYTES * values.length;
- int size = headerSize;
- for (byte[] value : values) {
- size += value.length;
- }
+ public void putDoubleMV(double[] values) {
+ putBytes(ArraySerDeUtils.serializeDoubleArrayWithLength(values));
+ }
- // Format : [numValues][length1][length2]...[lengthN][bytes1][bytes2]...[bytesN]
- byte[] serializedBytes = new byte[size];
- ByteBuffer byteBuffer = ByteBuffer.wrap(serializedBytes);
- byteBuffer.putInt(values.length);
- byteBuffer.position(headerSize);
- for (int i = 0; i < values.length; i++) {
- byteBuffer.putInt((i + 1) * Integer.BYTES, values[i].length);
- byteBuffer.put(values[i]);
- }
+ @Override
+ public void putStringMV(String[] values) {
+ putBytes(ArraySerDeUtils.serializeStringArray(values));
+ }
- putBytes(serializedBytes);
+ @Override
+ public void putBytesMV(byte[][] values) {
+ putBytes(ArraySerDeUtils.serializeBytesArray(values));
}
private void writeHugeChunk(byte[] bytes) {
@@ -287,7 +277,7 @@ private void write(ByteBuffer buffer, boolean huge) {
_chunkOffset += compressedSize;
_docIdOffset = _nextDocId;
} catch (IOException e) {
- LOGGER.error("Exception caught while compressing/writing data chunk", e);
+ _logger.error("Exception caught while compressing/writing data chunk", e);
throw new RuntimeException(e);
} finally {
if (mapped != null) {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV5.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV5.java
new file mode 100644
index 000000000000..b96812a05936
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV5.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.io.writer.impl;
+
+import java.io.File;
+import java.io.IOException;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.pinot.segment.local.utils.ArraySerDeUtils;
+import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
+
+
+/**
+ * Forward index writer that extends {@link VarByteChunkForwardIndexWriterV4} and overrides the data layout for
+ * multi-value fixed byte operations to improve space efficiency.
+ *
+ * Consider the following multi-value document as an example: {@code [int(1), int(2), int(3)]}.
+ * The current binary data layout in {@code VarByteChunkForwardIndexWriterV4} is as follows:
+ *
+ * 0x00000010 0x00000003 0x00000001 0x00000002 0x00000003
+ *
+ *
+ *
+ * - The first 4 bytes ({@code 0x00000010}) represent the total payload length of the byte array
+ * containing the multi-value document content, which in this case is 16 bytes.
+ *
+ * - The next 4 bytes ({@code 0x00000003}) represent the number of elements in the multi-value document (i.e., 3)
+ * .
+ *
+ * - The remaining 12 bytes ({@code 0x00000001 0x00000002 0x00000003}) represent the 3 integer values of the
+ * multi-value document: 1, 2, and 3.
+ *
+ *
+ * In Pinot, the fixed byte raw forward index can only store one specific fixed-length data type:
+ * {@code int}, {@code long}, {@code float}, or {@code double}. Instead of explicitly storing the number of elements
+ * for each document for multi-value document, this value can be inferred by:
+ *
+ * number of elements = buffer payload length / size of data type
+ *
+ *
+ * If the forward index uses the passthrough chunk compression type (i.e., no compression), we can save
+ * 4 bytes per document by omitting the explicit element count. This leads to the following space savings:
+ *
+ *
+ * - For documents with 0 elements, we save 50%.
+ * - For documents with 1 element, we save 33%.
+ * - For documents with 2 elements, we save 25%.
+ * - As the number of elements increases, the percentage of space saved decreases.
+ *
+ *
+ * For forward indexes that use compression to reduce data size, the savings can be even more significant
+ * in certain cases. This is demonstrated in the unit test {@link VarByteChunkV5Test#validateCompressionRatioIncrease},
+ * where ZStandard was used as the chunk compressor. In the test, 1 million short multi-value (MV) documents
+ * were inserted, following a Gaussian distribution for document lengths. Additionally, the values of each integer
+ * in the MV documents were somewhat repetitive. Under these conditions, we observed a 50%+ reduction in on-disk
+ * file size compared to the V4 forward index writer version.
+ *
+ * Note that the {@code VERSION} tag is a {@code static final} class variable set to {@code 5}. Since static
+ * variables are shadowed in the child class thus associated with the class that defines them, care must be taken to
+ * ensure that the parent class can correctly observe the child class's {@code VERSION} value at runtime. To handle
+ * this cleanly and correctly, the {@code getVersion()} method is overridden to return the concrete subclass's
+ * {@code VERSION} value, ensuring that the correct version number is returned even when using a reference to the
+ * parent class.
+ *
+ * @see VarByteChunkForwardIndexWriterV4
+ * @see VarByteChunkForwardIndexWriterV5#getVersion()
+ */
+@NotThreadSafe
+public class VarByteChunkForwardIndexWriterV5 extends VarByteChunkForwardIndexWriterV4 {
+ public static final int VERSION = 5;
+
+ public VarByteChunkForwardIndexWriterV5(File file, ChunkCompressionType compressionType, int chunkSize)
+ throws IOException {
+ super(file, compressionType, chunkSize);
+ }
+
+ // Override the parent class getVersion();
+ @Override
+ public int getVersion() {
+ return VERSION;
+ }
+
+ @Override
+ public void putIntMV(int[] values) {
+ putBytes(ArraySerDeUtils.serializeIntArrayWithoutLength(values));
+ }
+
+ @Override
+ public void putLongMV(long[] values) {
+ putBytes(ArraySerDeUtils.serializeLongArrayWithoutLength(values));
+ }
+
+ @Override
+ public void putFloatMV(float[] values) {
+ putBytes(ArraySerDeUtils.serializeFloatArrayWithoutLength(values));
+ }
+
+ @Override
+ public void putDoubleMV(double[] values) {
+ putBytes(ArraySerDeUtils.serializeDoubleArrayWithoutLength(values));
+ }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkWriter.java
index bf3537d67c98..82fbd2b4691e 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkWriter.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkWriter.java
@@ -29,6 +29,14 @@ public interface VarByteChunkWriter extends Closeable {
void putBytes(byte[] value);
+ void putIntMV(int[] values);
+
+ void putLongMV(long[] values);
+
+ void putFloatMV(float[] values);
+
+ void putDoubleMV(double[] values);
+
void putStringMV(String[] values);
void putBytesMV(byte[][] values);
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java
index 749c4cf704a0..b395e5b21cd8 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java
@@ -293,8 +293,9 @@ private boolean createDictionaryForColumn(ColumnIndexCreationInfo info, SegmentG
FieldIndexConfigs fieldIndexConfigs = config.getIndexConfigsByColName().get(column);
if (DictionaryIndexType.ignoreDictionaryOverride(config.isOptimizeDictionary(),
- config.isOptimizeDictionaryForMetrics(), config.getNoDictionarySizeRatioThreshold(), spec, fieldIndexConfigs,
- info.getDistinctValueCount(), info.getTotalNumberOfEntries())) {
+ config.isOptimizeDictionaryForMetrics(), config.getNoDictionarySizeRatioThreshold(),
+ config.getNoDictionaryCardinalityRatioThreshold(), spec, fieldIndexConfigs, info.getDistinctValueCount(),
+ info.getTotalNumberOfEntries())) {
// Ignore overrides and pick from config
createDictionary = info.isCreateDictionary();
}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java
index dcdcb9970516..b8a6bd6daafd 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java
@@ -20,9 +20,9 @@
import java.io.File;
import java.io.IOException;
-import java.nio.ByteBuffer;
import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriter;
import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4;
+import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV5;
import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkWriter;
import org.apache.pinot.segment.spi.V1Constants.Indexes;
import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
@@ -70,23 +70,32 @@ public MultiValueFixedByteRawIndexCreator(File indexFile, ChunkCompressionType c
ForwardIndexConfig.DEFAULT_TARGET_DOCS_PER_CHUNK);
}
-
public MultiValueFixedByteRawIndexCreator(File indexFile, ChunkCompressionType compressionType, int totalDocs,
DataType valueType, int maxNumberOfMultiValueElements, boolean deriveNumDocsPerChunk, int writerVersion,
int targetMaxChunkSizeBytes, int targetDocsPerChunk)
throws IOException {
- // Store the length followed by the values
- int totalMaxLength = Integer.BYTES + (maxNumberOfMultiValueElements * valueType.getStoredType().size());
if (writerVersion < VarByteChunkForwardIndexWriterV4.VERSION) {
+ // Store the length followed by the values
+ int totalMaxLength = Integer.BYTES + (maxNumberOfMultiValueElements * valueType.getStoredType().size());
int numDocsPerChunk = deriveNumDocsPerChunk ? Math.max(targetMaxChunkSizeBytes / (totalMaxLength
+ VarByteChunkForwardIndexWriter.CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE), 1) : targetDocsPerChunk;
_indexWriter =
new VarByteChunkForwardIndexWriter(indexFile, compressionType, totalDocs, numDocsPerChunk, totalMaxLength,
writerVersion);
} else {
- int chunkSize =
- ForwardIndexUtils.getDynamicTargetChunkSize(totalMaxLength, targetDocsPerChunk, targetMaxChunkSizeBytes);
- _indexWriter = new VarByteChunkForwardIndexWriterV4(indexFile, compressionType, chunkSize);
+ if (writerVersion == VarByteChunkForwardIndexWriterV5.VERSION) {
+ // Store only the values
+ int totalMaxLength = maxNumberOfMultiValueElements * valueType.getStoredType().size();
+ int chunkSize =
+ ForwardIndexUtils.getDynamicTargetChunkSize(totalMaxLength, targetDocsPerChunk, targetMaxChunkSizeBytes);
+ _indexWriter = new VarByteChunkForwardIndexWriterV5(indexFile, compressionType, chunkSize);
+ } else {
+ // Store the length followed by the values
+ int totalMaxLength = Integer.BYTES + (maxNumberOfMultiValueElements * valueType.getStoredType().size());
+ int chunkSize =
+ ForwardIndexUtils.getDynamicTargetChunkSize(totalMaxLength, targetDocsPerChunk, targetMaxChunkSizeBytes);
+ _indexWriter = new VarByteChunkForwardIndexWriterV4(indexFile, compressionType, chunkSize);
+ }
}
_valueType = valueType;
}
@@ -108,54 +117,22 @@ public DataType getValueType() {
@Override
public void putIntMV(int[] values) {
- byte[] bytes = new byte[Integer.BYTES + values.length * Integer.BYTES];
- ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
- //write the length
- byteBuffer.putInt(values.length);
- //write the content of each element
- for (int value : values) {
- byteBuffer.putInt(value);
- }
- _indexWriter.putBytes(bytes);
+ _indexWriter.putIntMV(values);
}
@Override
public void putLongMV(long[] values) {
- byte[] bytes = new byte[Integer.BYTES + values.length * Long.BYTES];
- ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
- //write the length
- byteBuffer.putInt(values.length);
- //write the content of each element
- for (long value : values) {
- byteBuffer.putLong(value);
- }
- _indexWriter.putBytes(bytes);
+ _indexWriter.putLongMV(values);
}
@Override
public void putFloatMV(float[] values) {
- byte[] bytes = new byte[Integer.BYTES + values.length * Float.BYTES];
- ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
- //write the length
- byteBuffer.putInt(values.length);
- //write the content of each element
- for (float value : values) {
- byteBuffer.putFloat(value);
- }
- _indexWriter.putBytes(bytes);
+ _indexWriter.putFloatMV(values);
}
@Override
public void putDoubleMV(double[] values) {
- byte[] bytes = new byte[Integer.BYTES + values.length * Double.BYTES];
- ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
- //write the length
- byteBuffer.putInt(values.length);
- //write the content of each element
- for (double value : values) {
- byteBuffer.putDouble(value);
- }
- _indexWriter.putBytes(bytes);
+ _indexWriter.putDoubleMV(values);
}
@Override
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexType.java
index 9df6d24daaad..9fb6dbce87cc 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexType.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexType.java
@@ -218,11 +218,12 @@ public static boolean shouldUseVarLengthDictionary(FieldSpec.DataType columnStor
* This function evaluates whether to override dictionary (i.e use noDictionary)
* for a column even when its explicitly configured. This evaluation is for both dimension and metric
* column types.
+ *
+ * @return true if dictionary should be created, false if noDictionary should be used
*/
- public static boolean ignoreDictionaryOverride(boolean optimizeDictionary,
- boolean optimizeDictionaryForMetrics, double noDictionarySizeRatioThreshold,
- FieldSpec fieldSpec, FieldIndexConfigs fieldIndexConfigs, int cardinality,
- int totalNumberOfEntries) {
+ public static boolean ignoreDictionaryOverride(boolean optimizeDictionary, boolean optimizeDictionaryForMetrics,
+ double noDictionarySizeRatioThreshold, @Nullable Double noDictionaryCardinalityRatioThreshold,
+ FieldSpec fieldSpec, FieldIndexConfigs fieldIndexConfigs, int cardinality, int totalNumberOfEntries) {
// For an inverted index dictionary is required
if (fieldIndexConfigs.getConfig(StandardIndexes.inverted()).isEnabled()) {
return true;
@@ -236,22 +237,38 @@ public static boolean ignoreDictionaryOverride(boolean optimizeDictionary,
// Do not create dictionary if index size with dictionary is going to be larger than index size without dictionary
// This is done to reduce the cost of dictionary for high cardinality columns
// Off by default and needs optimizeDictionary to be set to true
- if (fieldSpec.isSingleValueField() && fieldSpec.getDataType().isFixedWidth()) {
- // if you can safely enable dictionary, you can ignore overrides
- return canSafelyCreateDictionaryWithinThreshold(cardinality, totalNumberOfEntries,
- noDictionarySizeRatioThreshold, fieldSpec);
+ if (fieldSpec.isSingleValueField()) {
+ return ignoreDictionaryOverrideForSingleValueFields(cardinality, totalNumberOfEntries,
+ noDictionarySizeRatioThreshold, noDictionaryCardinalityRatioThreshold, fieldSpec);
}
}
+ if (optimizeDictionaryForMetrics && !optimizeDictionary && fieldSpec.isSingleValueField()
+ && fieldSpec.getFieldType() == FieldSpec.FieldType.METRIC) {
+ return ignoreDictionaryOverrideForSingleValueFields(cardinality, totalNumberOfEntries,
+ noDictionarySizeRatioThreshold, noDictionaryCardinalityRatioThreshold, fieldSpec);
+ }
+ return true;
+ }
- if (optimizeDictionaryForMetrics && !optimizeDictionary) {
- if (fieldSpec.isSingleValueField() && fieldSpec.getDataType().isFixedWidth() && fieldSpec.getFieldType()
- == FieldSpec.FieldType.METRIC) {
+ /**
+ * Hold common logic for ignoring dictionary override for single value fields, used for dim and metric cols
+ */
+ private static boolean ignoreDictionaryOverrideForSingleValueFields(int cardinality, int totalNumberOfEntries,
+ double noDictionarySizeRatioThreshold, Double noDictionaryCardinalityRatioThreshold, FieldSpec fieldSpec) {
+ if (fieldSpec.isSingleValueField()) {
+ if (fieldSpec.getDataType().isFixedWidth()) {
// if you can safely enable dictionary, you can ignore overrides
return canSafelyCreateDictionaryWithinThreshold(cardinality, totalNumberOfEntries,
noDictionarySizeRatioThreshold, fieldSpec);
}
+ // Config not set, default to old behavior of create dictionary for var width cols
+ if (noDictionaryCardinalityRatioThreshold == null) {
+ return true;
+ }
+ // Variable width type, so create based simply on cardinality threshold since size cannot be calculated easily
+ return noDictionaryCardinalityRatioThreshold * totalNumberOfEntries > cardinality;
}
- return true;
+ return false;
}
/**
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java
index db815761d9ea..cc7201ed985f 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java
@@ -21,6 +21,7 @@
import java.util.Arrays;
import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4;
+import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV5;
import org.apache.pinot.segment.local.segment.creator.impl.fwd.CLPForwardIndexCreatorV1;
import org.apache.pinot.segment.local.segment.index.readers.forward.CLPForwardIndexReaderV1;
import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBitMVEntryDictForwardIndexReader;
@@ -30,6 +31,7 @@
import org.apache.pinot.segment.local.segment.index.readers.forward.FixedByteChunkSVForwardIndexReader;
import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBytePower2ChunkSVForwardIndexReader;
import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkForwardIndexReaderV4;
+import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkForwardIndexReaderV5;
import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkMVForwardIndexReader;
import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkSVForwardIndexReader;
import org.apache.pinot.segment.local.segment.index.readers.sorted.SortedIndexReaderImpl;
@@ -106,7 +108,10 @@ public static ForwardIndexReader createRawIndexReader(PinotDataBuffer dataBuffer
: new FixedByteChunkSVForwardIndexReader(dataBuffer, storedType);
}
- if (version == VarByteChunkForwardIndexWriterV4.VERSION) {
+ if (version == VarByteChunkForwardIndexWriterV5.VERSION) {
+ // V5 is the same as V4 except the multi-value docs have implicit value count rather than explicit
+ return new VarByteChunkForwardIndexReaderV5(dataBuffer, storedType, isSingleValue);
+ } else if (version == VarByteChunkForwardIndexWriterV4.VERSION) {
// V4 reader is common for sv var byte, mv fixed byte and mv var byte
return new VarByteChunkForwardIndexReaderV4(dataBuffer, storedType, isSingleValue);
} else {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java
index 3b654a7ebc59..21dd9e18c250 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java
@@ -303,9 +303,10 @@ Map> computeOperations(SegmentDirectory.Reader segmentRe
if (existingColumnMetadata.getFieldSpec().getFieldType() != FieldSpec.FieldType.COMPLEX
&& DictionaryIndexType.ignoreDictionaryOverride(_tableConfig.getIndexingConfig().isOptimizeDictionary(),
_tableConfig.getIndexingConfig().isOptimizeDictionaryForMetrics(),
- _tableConfig.getIndexingConfig().getNoDictionarySizeRatioThreshold(), existingColumnMetadata.getFieldSpec(),
- _fieldIndexConfigs.get(column), existingColumnMetadata.getCardinality(),
- existingColumnMetadata.getTotalNumberOfEntries())) {
+ _tableConfig.getIndexingConfig().getNoDictionarySizeRatioThreshold(),
+ _tableConfig.getIndexingConfig().getNoDictionaryCardinalityRatioThreshold(),
+ existingColumnMetadata.getFieldSpec(), _fieldIndexConfigs.get(column),
+ existingColumnMetadata.getCardinality(), existingColumnMetadata.getTotalNumberOfEntries())) {
columnOperationsMap.put(column, Collections.singletonList(Operation.ENABLE_DICTIONARY));
}
} else if (existingHasDict && !newIsDict) {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java
index 0684fe9097d5..51e33e544453 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java
@@ -83,7 +83,6 @@ public class IndexLoadingConfig {
private final Map _noDictionaryConfig = new HashMap<>();
private final Set _varLengthDictionaryColumns = new HashSet<>();
private Set _onHeapDictionaryColumns = new HashSet<>();
- private final Set _forwardIndexDisabledColumns = new HashSet<>();
private Map _bloomFilterConfigs = new HashMap<>();
private boolean _enableDynamicStarTreeCreation;
private List _starTreeIndexConfigs;
@@ -540,10 +539,6 @@ public Set getOnHeapDictionaryColumns() {
return unmodifiable(_onHeapDictionaryColumns);
}
- public Set getForwardIndexDisabledColumns() {
- return unmodifiable(_forwardIndexDisabledColumns);
- }
-
public Map getBloomFilterConfigs() {
return unmodifiable(_bloomFilterConfigs);
}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
index e655f9e470b9..f6dc1a159b35 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
@@ -411,7 +411,7 @@ protected boolean createColumnV1Indices(String column)
}
// TODO: Support forward index disabled derived column
- if (_indexLoadingConfig.getForwardIndexDisabledColumns().contains(column)) {
+ if (isForwardIndexDisabled(column)) {
LOGGER.warn("Skip creating forward index disabled derived column: {}", column);
if (errorOnFailure) {
throw new UnsupportedOperationException(
@@ -443,8 +443,8 @@ protected boolean createColumnV1Indices(String column)
* Check and return whether the forward index is disabled for a given column
*/
protected boolean isForwardIndexDisabled(String column) {
- return _indexLoadingConfig.getForwardIndexDisabledColumns() != null
- && _indexLoadingConfig.getForwardIndexDisabledColumns().contains(column);
+ FieldIndexConfigs fieldIndexConfig = _indexLoadingConfig.getFieldIndexConfig(column);
+ return fieldIndexConfig != null && fieldIndexConfig.getConfig(StandardIndexes.forward()).isDisabled();
}
/**
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/FixedByteChunkMVForwardIndexReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/FixedByteChunkMVForwardIndexReader.java
index f96ed6e878a6..8e53ecb15639 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/FixedByteChunkMVForwardIndexReader.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/FixedByteChunkMVForwardIndexReader.java
@@ -22,6 +22,7 @@
import java.util.List;
import javax.annotation.Nullable;
import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriter;
+import org.apache.pinot.segment.local.utils.ArraySerDeUtils;
import org.apache.pinot.segment.spi.memory.PinotDataBuffer;
import org.apache.pinot.spi.data.FieldSpec.DataType;
@@ -53,92 +54,47 @@ public ChunkReaderContext createContext() {
@Override
public int getIntMV(int docId, int[] valueBuffer, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- int numValues = byteBuffer.getInt();
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getInt();
- }
- return numValues;
+ return ArraySerDeUtils.deserializeIntArrayWithLength(slice(docId, context), valueBuffer);
}
@Override
public int[] getIntMV(int docId, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- int numValues = byteBuffer.getInt();
- int[] valueBuffer = new int[numValues];
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getInt();
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeIntArrayWithLength(slice(docId, context));
}
@Override
public int getLongMV(int docId, long[] valueBuffer, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- int numValues = byteBuffer.getInt();
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getLong();
- }
- return numValues;
+ return ArraySerDeUtils.deserializeLongArrayWithLength(slice(docId, context), valueBuffer);
}
@Override
public long[] getLongMV(int docId, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- int numValues = byteBuffer.getInt();
- long[] valueBuffer = new long[numValues];
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getLong();
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeLongArrayWithLength(slice(docId, context));
}
@Override
public int getFloatMV(int docId, float[] valueBuffer, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- int numValues = byteBuffer.getInt();
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getFloat();
- }
- return numValues;
+ return ArraySerDeUtils.deserializeFloatArrayWithLength(slice(docId, context), valueBuffer);
}
@Override
public float[] getFloatMV(int docId, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- int numValues = byteBuffer.getInt();
- float[] valueBuffer = new float[numValues];
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getFloat();
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeFloatArrayWithLength(slice(docId, context));
}
@Override
public int getDoubleMV(int docId, double[] valueBuffer, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- int numValues = byteBuffer.getInt();
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getDouble();
- }
- return numValues;
+ return ArraySerDeUtils.deserializeDoubleArrayWithLength(slice(docId, context), valueBuffer);
}
@Override
public double[] getDoubleMV(int docId, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- int numValues = byteBuffer.getInt();
- double[] valueBuffer = new double[numValues];
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getDouble();
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeDoubleArrayWithLength(slice(docId, context));
}
@Override
public int getNumValuesMV(int docId, ChunkReaderContext context) {
- ByteBuffer byteBuffer = slice(docId, context);
- return byteBuffer.getInt();
+ return slice(docId, context).getInt();
}
private ByteBuffer slice(int docId, ChunkReaderContext context) {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV4.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV4.java
index 851f5f3e7d7d..cf2a8b4de4da 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV4.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV4.java
@@ -29,6 +29,7 @@
import java.util.Map;
import org.apache.pinot.segment.local.io.compression.ChunkCompressorFactory;
import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4;
+import org.apache.pinot.segment.local.utils.ArraySerDeUtils;
import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
import org.apache.pinot.segment.spi.compression.ChunkDecompressor;
import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
@@ -46,6 +47,8 @@
* Chunk-based raw (non-dictionary-encoded) forward index reader for values of SV variable length data types
* (BIG_DECIMAL, STRING, BYTES), MV fixed length and MV variable length data types.
* For data layout, please refer to the documentation for {@link VarByteChunkForwardIndexWriterV4}
+ *
+ * TODO: Consider reading directly from sliced ByteBuffer instead of copying to byte[] first
*/
public class VarByteChunkForwardIndexReaderV4
implements ForwardIndexReader {
@@ -64,8 +67,7 @@ public class VarByteChunkForwardIndexReaderV4
public VarByteChunkForwardIndexReaderV4(PinotDataBuffer dataBuffer, FieldSpec.DataType storedType,
boolean isSingleValue) {
- int version = dataBuffer.getInt(0);
- Preconditions.checkState(version == VarByteChunkForwardIndexWriterV4.VERSION, "Illegal index version: %s", version);
+ validateIndexVersion(dataBuffer);
_storedType = storedType;
_targetDecompressedChunkSize = dataBuffer.getInt(4);
_chunkCompressionType = ChunkCompressionType.valueOf(dataBuffer.getInt(8));
@@ -78,6 +80,15 @@ public VarByteChunkForwardIndexReaderV4(PinotDataBuffer dataBuffer, FieldSpec.Da
_isSingleValue = isSingleValue;
}
+ public void validateIndexVersion(PinotDataBuffer dataBuffer) {
+ int version = dataBuffer.getInt(0);
+ Preconditions.checkState(version == getVersion(), "Illegal index version: %s", version);
+ }
+
+ public int getVersion() {
+ return VarByteChunkForwardIndexWriterV4.VERSION;
+ }
+
@Override
public boolean isDictionaryEncoded() {
return false;
@@ -130,144 +141,62 @@ public Map getMap(int docId, ReaderContext context) {
@Override
public int getIntMV(int docId, int[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getInt();
- }
- return numValues;
+ return ArraySerDeUtils.deserializeIntArrayWithLength(context.getValue(docId), valueBuffer);
}
@Override
public int[] getIntMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- int[] valueBuffer = new int[numValues];
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getInt();
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeIntArrayWithLength(context.getValue(docId));
}
@Override
public int getLongMV(int docId, long[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getLong();
- }
- return numValues;
+ return ArraySerDeUtils.deserializeLongArrayWithLength(context.getValue(docId), valueBuffer);
}
@Override
public long[] getLongMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- long[] valueBuffer = new long[numValues];
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getLong();
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeLongArrayWithLength(context.getValue(docId));
}
@Override
public int getFloatMV(int docId, float[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getFloat();
- }
- return numValues;
+ return ArraySerDeUtils.deserializeFloatArrayWithLength(context.getValue(docId), valueBuffer);
}
@Override
public float[] getFloatMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- float[] valueBuffer = new float[numValues];
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getFloat();
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeFloatArrayWithLength(context.getValue(docId));
}
@Override
public int getDoubleMV(int docId, double[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getDouble();
- }
- return numValues;
+ return ArraySerDeUtils.deserializeDoubleArrayWithLength(context.getValue(docId), valueBuffer);
}
@Override
public double[] getDoubleMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- double[] valueBuffer = new double[numValues];
- for (int i = 0; i < numValues; i++) {
- valueBuffer[i] = byteBuffer.getFloat();
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeDoubleArrayWithLength(context.getValue(docId));
}
@Override
public int getStringMV(int docId, String[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- byte[] bytes = context.getValue(docId);
- ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
- int numValues = byteBuffer.getInt();
- int offset = (numValues + 1) * Integer.BYTES;
- for (int i = 0; i < numValues; i++) {
- int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
- valueBuffer[i] = new String(bytes, offset, length, StandardCharsets.UTF_8);
- offset += length;
- }
- return numValues;
+ return ArraySerDeUtils.deserializeStringArray(context.getValue(docId), valueBuffer);
}
@Override
public String[] getStringMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- byte[] bytes = context.getValue(docId);
- ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
- int numValues = byteBuffer.getInt();
- int offset = (numValues + 1) * Integer.BYTES;
- String[] valueBuffer = new String[numValues];
- for (int i = 0; i < numValues; i++) {
- int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
- valueBuffer[i] = new String(bytes, offset, length, StandardCharsets.UTF_8);
- offset += length;
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeStringArray(context.getValue(docId));
}
@Override
public int getBytesMV(int docId, byte[][] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- byteBuffer.position((numValues + 1) * Integer.BYTES);
- for (int i = 0; i < numValues; i++) {
- int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
- byte[] bytes = new byte[length];
- byteBuffer.get(bytes, 0, length);
- valueBuffer[i] = bytes;
- }
- return numValues;
+ return ArraySerDeUtils.deserializeBytesArray(context.getValue(docId), valueBuffer);
}
@Override
public byte[][] getBytesMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
- ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId));
- int numValues = byteBuffer.getInt();
- byteBuffer.position((numValues + 1) * Integer.BYTES);
- byte[][] valueBuffer = new byte[numValues][];
- for (int i = 0; i < numValues; i++) {
- int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
- byte[] bytes = new byte[length];
- byteBuffer.get(bytes, 0, length);
- valueBuffer[i] = bytes;
- }
- return valueBuffer;
+ return ArraySerDeUtils.deserializeBytesArray(context.getValue(docId));
}
@Override
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV5.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV5.java
new file mode 100644
index 000000000000..e72fedfc584e
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV5.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.segment.index.readers.forward;
+
+import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4;
+import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV5;
+import org.apache.pinot.segment.local.utils.ArraySerDeUtils;
+import org.apache.pinot.segment.spi.memory.PinotDataBuffer;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * Chunk-based raw (non-dictionary-encoded) forward index reader for values of SV variable length data types
+ * (BIG_DECIMAL, STRING, BYTES), MV fixed length and MV variable length data types.
+ * For data layout, please refer to the documentation for {@link VarByteChunkForwardIndexWriterV4}
+ */
+public class VarByteChunkForwardIndexReaderV5 extends VarByteChunkForwardIndexReaderV4 {
+ public VarByteChunkForwardIndexReaderV5(PinotDataBuffer dataBuffer, FieldSpec.DataType storedType,
+ boolean isSingleValue) {
+ super(dataBuffer, storedType, isSingleValue);
+ }
+
+ @Override
+ public int getVersion() {
+ return VarByteChunkForwardIndexWriterV5.VERSION;
+ }
+
+ @Override
+ public int getIntMV(int docId, int[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
+ return ArraySerDeUtils.deserializeIntArrayWithoutLength(context.getValue(docId), valueBuffer);
+ }
+
+ @Override
+ public int[] getIntMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
+ return ArraySerDeUtils.deserializeIntArrayWithoutLength(context.getValue(docId));
+ }
+
+ @Override
+ public int getLongMV(int docId, long[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
+ return ArraySerDeUtils.deserializeLongArrayWithoutLength(context.getValue(docId), valueBuffer);
+ }
+
+ @Override
+ public long[] getLongMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
+ return ArraySerDeUtils.deserializeLongArrayWithoutLength(context.getValue(docId));
+ }
+
+ @Override
+ public int getFloatMV(int docId, float[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
+ return ArraySerDeUtils.deserializeFloatArrayWithoutLength(context.getValue(docId), valueBuffer);
+ }
+
+ @Override
+ public float[] getFloatMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
+ return ArraySerDeUtils.deserializeFloatArrayWithoutLength(context.getValue(docId));
+ }
+
+ @Override
+ public int getDoubleMV(int docId, double[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
+ return ArraySerDeUtils.deserializeDoubleArrayWithoutLength(context.getValue(docId), valueBuffer);
+ }
+
+ @Override
+ public double[] getDoubleMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) {
+ return ArraySerDeUtils.deserializeDoubleArrayWithoutLength(context.getValue(docId));
+ }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkMVForwardIndexReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkMVForwardIndexReader.java
index 1d133bb896c3..bd7dc5dbb746 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkMVForwardIndexReader.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkMVForwardIndexReader.java
@@ -19,10 +19,10 @@
package org.apache.pinot.segment.local.segment.index.readers.forward;
import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
import java.util.List;
import javax.annotation.Nullable;
import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriter;
+import org.apache.pinot.segment.local.utils.ArraySerDeUtils;
import org.apache.pinot.segment.spi.memory.PinotDataBuffer;
import org.apache.pinot.spi.data.FieldSpec.DataType;
@@ -31,6 +31,8 @@
* Chunk-based multi-value raw (non-dictionary-encoded) forward index reader for values of variable length data type
* (STRING, BYTES).
*
For data layout, please refer to the documentation for {@link VarByteChunkForwardIndexWriter}
+ *
+ * TODO: Consider reading directly from sliced ByteBuffer instead of copying to byte[] first
*/
public final class VarByteChunkMVForwardIndexReader extends BaseChunkForwardIndexReader {
private static final int ROW_OFFSET_SIZE = VarByteChunkForwardIndexWriter.CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE;
@@ -53,105 +55,28 @@ public ChunkReaderContext createContext() {
}
@Override
- public int getStringMV(final int docId, final String[] valueBuffer, final ChunkReaderContext context) {
- byte[] compressedBytes;
- if (_isCompressed) {
- compressedBytes = getBytesCompressed(docId, context);
- } else {
- compressedBytes = getBytesUncompressed(docId);
- }
- ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes);
- int numValues = byteBuffer.getInt();
- int contentOffset = (numValues + 1) * Integer.BYTES;
- for (int i = 0; i < numValues; i++) {
- int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
- byte[] bytes = new byte[length];
- byteBuffer.position(contentOffset);
- byteBuffer.get(bytes, 0, length);
- valueBuffer[i] = new String(bytes, StandardCharsets.UTF_8);
- contentOffset += length;
- }
- return numValues;
+ public int getStringMV(int docId, String[] valueBuffer, ChunkReaderContext context) {
+ return ArraySerDeUtils.deserializeStringArray(getBytes(docId, context), valueBuffer);
}
@Override
- public String[] getStringMV(final int docId, final ChunkReaderContext context) {
- byte[] compressedBytes;
- if (_isCompressed) {
- compressedBytes = getBytesCompressed(docId, context);
- } else {
- compressedBytes = getBytesUncompressed(docId);
- }
- ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes);
- int numValues = byteBuffer.getInt();
- int contentOffset = (numValues + 1) * Integer.BYTES;
- String[] valueBuffer = new String[numValues];
- for (int i = 0; i < numValues; i++) {
- int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
- byte[] bytes = new byte[length];
- byteBuffer.position(contentOffset);
- byteBuffer.get(bytes, 0, length);
- valueBuffer[i] = new String(bytes, StandardCharsets.UTF_8);
- contentOffset += length;
- }
- return valueBuffer;
+ public String[] getStringMV(int docId, ChunkReaderContext context) {
+ return ArraySerDeUtils.deserializeStringArray(getBytes(docId, context));
}
@Override
- public int getBytesMV(final int docId, final byte[][] valueBuffer, final ChunkReaderContext context) {
- byte[] compressedBytes;
- if (_isCompressed) {
- compressedBytes = getBytesCompressed(docId, context);
- } else {
- compressedBytes = getBytesUncompressed(docId);
- }
- ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes);
- int numValues = byteBuffer.getInt();
- int contentOffset = (numValues + 1) * Integer.BYTES;
- for (int i = 0; i < numValues; i++) {
- int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
- byte[] bytes = new byte[length];
- byteBuffer.position(contentOffset);
- byteBuffer.get(bytes, 0, length);
- valueBuffer[i] = bytes;
- contentOffset += length;
- }
- return numValues;
+ public int getBytesMV(int docId, byte[][] valueBuffer, ChunkReaderContext context) {
+ return ArraySerDeUtils.deserializeBytesArray(getBytes(docId, context), valueBuffer);
}
@Override
- public byte[][] getBytesMV(final int docId, final ChunkReaderContext context) {
- byte[] compressedBytes;
- if (_isCompressed) {
- compressedBytes = getBytesCompressed(docId, context);
- } else {
- compressedBytes = getBytesUncompressed(docId);
- }
- ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes);
- int numValues = byteBuffer.getInt();
- int contentOffset = (numValues + 1) * Integer.BYTES;
- byte[][] valueBuffer = new byte[numValues][];
- for (int i = 0; i < numValues; i++) {
- int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
- byte[] bytes = new byte[length];
- byteBuffer.position(contentOffset);
- byteBuffer.get(bytes, 0, length);
- valueBuffer[i] = bytes;
- contentOffset += length;
- }
- return valueBuffer;
+ public byte[][] getBytesMV(int docId, ChunkReaderContext context) {
+ return ArraySerDeUtils.deserializeBytesArray(getBytes(docId, context));
}
@Override
- public int getNumValuesMV(final int docId, final ChunkReaderContext context) {
- byte[] compressedBytes;
- if (_isCompressed) {
- compressedBytes = getBytesCompressed(docId, context);
- } else {
- compressedBytes = getBytesUncompressed(docId);
- }
- ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes);
- return byteBuffer.getInt();
+ public int getNumValuesMV(int docId, ChunkReaderContext context) {
+ return ByteBuffer.wrap(getBytes(docId, context)).getInt();
}
@Override
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ArraySerDeUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ArraySerDeUtils.java
new file mode 100644
index 000000000000..58238a33e068
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ArraySerDeUtils.java
@@ -0,0 +1,363 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.utils;
+
+import java.nio.ByteBuffer;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+
+public class ArraySerDeUtils {
+ private ArraySerDeUtils() {
+ }
+
+ public static byte[] serializeIntArrayWithLength(int[] values) {
+ byte[] bytes = new byte[Integer.BYTES + values.length * Integer.BYTES];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ byteBuffer.putInt(values.length);
+ writeValues(byteBuffer, values);
+ return bytes;
+ }
+
+ public static int[] deserializeIntArrayWithLength(byte[] bytes) {
+ return deserializeIntArrayWithLength(ByteBuffer.wrap(bytes));
+ }
+
+ public static int[] deserializeIntArrayWithLength(ByteBuffer byteBuffer) {
+ int length = byteBuffer.getInt();
+ int[] values = new int[length];
+ readValues(byteBuffer, values, length);
+ return values;
+ }
+
+ public static int deserializeIntArrayWithLength(byte[] bytes, int[] values) {
+ return deserializeIntArrayWithLength(ByteBuffer.wrap(bytes), values);
+ }
+
+ public static int deserializeIntArrayWithLength(ByteBuffer byteBuffer, int[] values) {
+ int length = byteBuffer.getInt();
+ readValues(byteBuffer, values, length);
+ return length;
+ }
+
+ public static byte[] serializeIntArrayWithoutLength(int[] values) {
+ byte[] bytes = new byte[values.length * Integer.BYTES];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ writeValues(byteBuffer, values);
+ return bytes;
+ }
+
+ public static int[] deserializeIntArrayWithoutLength(byte[] bytes) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int length = bytes.length / Integer.BYTES;
+ int[] values = new int[length];
+ readValues(byteBuffer, values, length);
+ return values;
+ }
+
+ public static int deserializeIntArrayWithoutLength(byte[] bytes, int[] values) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int length = bytes.length / Integer.BYTES;
+ readValues(byteBuffer, values, length);
+ return length;
+ }
+
+ private static void writeValues(ByteBuffer byteBuffer, int[] values) {
+ for (int value : values) {
+ byteBuffer.putInt(value);
+ }
+ }
+
+ private static void readValues(ByteBuffer byteBuffer, int[] values, int length) {
+ for (int i = 0; i < length; i++) {
+ values[i] = byteBuffer.getInt();
+ }
+ }
+
+ public static byte[] serializeLongArrayWithLength(long[] values) {
+ byte[] bytes = new byte[Integer.BYTES + values.length * Long.BYTES];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ byteBuffer.putInt(values.length);
+ writeValues(byteBuffer, values);
+ return bytes;
+ }
+
+ public static long[] deserializeLongArrayWithLength(byte[] bytes) {
+ return deserializeLongArrayWithLength(ByteBuffer.wrap(bytes));
+ }
+
+ public static long[] deserializeLongArrayWithLength(ByteBuffer byteBuffer) {
+ int length = byteBuffer.getInt();
+ long[] values = new long[length];
+ readValues(byteBuffer, values, length);
+ return values;
+ }
+
+ public static int deserializeLongArrayWithLength(byte[] bytes, long[] values) {
+ return deserializeLongArrayWithLength(ByteBuffer.wrap(bytes), values);
+ }
+
+ public static int deserializeLongArrayWithLength(ByteBuffer byteBuffer, long[] values) {
+ int length = byteBuffer.getInt();
+ readValues(byteBuffer, values, length);
+ return length;
+ }
+
+ public static byte[] serializeLongArrayWithoutLength(long[] values) {
+ byte[] bytes = new byte[values.length * Long.BYTES];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ writeValues(byteBuffer, values);
+ return bytes;
+ }
+
+ public static long[] deserializeLongArrayWithoutLength(byte[] bytes) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int length = bytes.length / Long.BYTES;
+ long[] values = new long[length];
+ readValues(byteBuffer, values, length);
+ return values;
+ }
+
+ public static int deserializeLongArrayWithoutLength(byte[] bytes, long[] values) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int length = bytes.length / Long.BYTES;
+ readValues(byteBuffer, values, length);
+ return length;
+ }
+
+ private static void writeValues(ByteBuffer byteBuffer, long[] values) {
+ for (long value : values) {
+ byteBuffer.putLong(value);
+ }
+ }
+
+ private static void readValues(ByteBuffer byteBuffer, long[] values, int length) {
+ for (int i = 0; i < length; i++) {
+ values[i] = byteBuffer.getLong();
+ }
+ }
+
+ public static byte[] serializeFloatArrayWithLength(float[] values) {
+ byte[] bytes = new byte[Integer.BYTES + values.length * Float.BYTES];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ byteBuffer.putInt(values.length);
+ writeValues(byteBuffer, values);
+ return bytes;
+ }
+
+ public static float[] deserializeFloatArrayWithLength(byte[] bytes) {
+ return deserializeFloatArrayWithLength(ByteBuffer.wrap(bytes));
+ }
+
+ public static float[] deserializeFloatArrayWithLength(ByteBuffer byteBuffer) {
+ int length = byteBuffer.getInt();
+ float[] values = new float[length];
+ readValues(byteBuffer, values, length);
+ return values;
+ }
+
+ public static int deserializeFloatArrayWithLength(byte[] bytes, float[] values) {
+ return deserializeFloatArrayWithLength(ByteBuffer.wrap(bytes), values);
+ }
+
+ public static int deserializeFloatArrayWithLength(ByteBuffer byteBuffer, float[] values) {
+ int length = byteBuffer.getInt();
+ readValues(byteBuffer, values, length);
+ return length;
+ }
+
+ public static byte[] serializeFloatArrayWithoutLength(float[] values) {
+ byte[] bytes = new byte[values.length * Float.BYTES];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ writeValues(byteBuffer, values);
+ return bytes;
+ }
+
+ public static float[] deserializeFloatArrayWithoutLength(byte[] bytes) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int length = bytes.length / Float.BYTES;
+ float[] values = new float[length];
+ readValues(byteBuffer, values, length);
+ return values;
+ }
+
+ public static int deserializeFloatArrayWithoutLength(byte[] bytes, float[] values) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int length = bytes.length / Float.BYTES;
+ readValues(byteBuffer, values, length);
+ return length;
+ }
+
+ private static void writeValues(ByteBuffer byteBuffer, float[] values) {
+ for (float value : values) {
+ byteBuffer.putFloat(value);
+ }
+ }
+
+ private static void readValues(ByteBuffer byteBuffer, float[] values, int length) {
+ for (int i = 0; i < length; i++) {
+ values[i] = byteBuffer.getFloat();
+ }
+ }
+
+ public static byte[] serializeDoubleArrayWithLength(double[] values) {
+ byte[] bytes = new byte[Integer.BYTES + values.length * Double.BYTES];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ byteBuffer.putInt(values.length);
+ writeValues(byteBuffer, values);
+ return bytes;
+ }
+
+ public static double[] deserializeDoubleArrayWithLength(byte[] bytes) {
+ return deserializeDoubleArrayWithLength(ByteBuffer.wrap(bytes));
+ }
+
+ public static double[] deserializeDoubleArrayWithLength(ByteBuffer byteBuffer) {
+ int length = byteBuffer.getInt();
+ double[] values = new double[length];
+ readValues(byteBuffer, values, length);
+ return values;
+ }
+
+ public static int deserializeDoubleArrayWithLength(byte[] bytes, double[] values) {
+ return deserializeDoubleArrayWithLength(ByteBuffer.wrap(bytes), values);
+ }
+
+ public static int deserializeDoubleArrayWithLength(ByteBuffer byteBuffer, double[] values) {
+ int length = byteBuffer.getInt();
+ readValues(byteBuffer, values, length);
+ return length;
+ }
+
+ public static byte[] serializeDoubleArrayWithoutLength(double[] values) {
+ byte[] bytes = new byte[values.length * Double.BYTES];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ writeValues(byteBuffer, values);
+ return bytes;
+ }
+
+ public static double[] deserializeDoubleArrayWithoutLength(byte[] bytes) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int length = bytes.length / Double.BYTES;
+ double[] values = new double[length];
+ readValues(byteBuffer, values, length);
+ return values;
+ }
+
+ public static int deserializeDoubleArrayWithoutLength(byte[] bytes, double[] values) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int length = bytes.length / Double.BYTES;
+ readValues(byteBuffer, values, length);
+ return length;
+ }
+
+ private static void writeValues(ByteBuffer byteBuffer, double[] values) {
+ for (double value : values) {
+ byteBuffer.putDouble(value);
+ }
+ }
+
+ private static void readValues(ByteBuffer byteBuffer, double[] values, int length) {
+ for (int i = 0; i < length; i++) {
+ values[i] = byteBuffer.getDouble();
+ }
+ }
+
+ public static byte[] serializeStringArray(String[] values) {
+ // Format: [numValues][length1][length2]...[lengthN][value1][value2]...[valueN]
+ int headerSize = Integer.BYTES + Integer.BYTES * values.length;
+ int size = headerSize;
+ byte[][] stringBytes = new byte[values.length][];
+ for (int i = 0; i < values.length; i++) {
+ stringBytes[i] = values[i].getBytes(UTF_8);
+ size += stringBytes[i].length;
+ }
+ return writeValues(stringBytes, size, headerSize);
+ }
+
+ public static String[] deserializeStringArray(byte[] bytes) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int numValues = byteBuffer.getInt();
+ String[] values = new String[numValues];
+ readValues(bytes, byteBuffer, values, numValues);
+ return values;
+ }
+
+ public static int deserializeStringArray(byte[] bytes, String[] values) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int numValues = byteBuffer.getInt();
+ readValues(bytes, byteBuffer, values, numValues);
+ return numValues;
+ }
+
+ public static byte[] serializeBytesArray(byte[][] values) {
+ // Format: [numValues][length1][length2]...[lengthN][value1][value2]...[valueN]
+ int headerSize = Integer.BYTES + Integer.BYTES * values.length;
+ int size = headerSize;
+ for (byte[] value : values) {
+ size += value.length;
+ }
+ return writeValues(values, size, headerSize);
+ }
+
+ public static byte[][] deserializeBytesArray(byte[] bytes) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int numValues = byteBuffer.getInt();
+ byte[][] values = new byte[numValues][];
+ readValues(byteBuffer, values, numValues);
+ return values;
+ }
+
+ public static int deserializeBytesArray(byte[] bytes, byte[][] values) {
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ int numValues = byteBuffer.getInt();
+ readValues(byteBuffer, values, numValues);
+ return numValues;
+ }
+
+ private static byte[] writeValues(byte[][] values, int size, int headerSize) {
+ byte[] bytes = new byte[size];
+ ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
+ byteBuffer.putInt(values.length);
+ byteBuffer.position(headerSize);
+ for (int i = 0; i < values.length; i++) {
+ byteBuffer.putInt((i + 1) * Integer.BYTES, values[i].length);
+ byteBuffer.put(values[i]);
+ }
+ return bytes;
+ }
+
+ private static void readValues(byte[] bytes, ByteBuffer byteBuffer, String[] values, int numValues) {
+ int offset = (numValues + 1) * Integer.BYTES;
+ for (int i = 0; i < numValues; i++) {
+ int length = byteBuffer.getInt();
+ values[i] = new String(bytes, offset, length, UTF_8);
+ offset += length;
+ }
+ }
+
+ private static void readValues(ByteBuffer byteBuffer, byte[][] values, int numValues) {
+ byteBuffer.position((numValues + 1) * Integer.BYTES);
+ for (int i = 0; i < numValues; i++) {
+ int length = byteBuffer.getInt((i + 1) * Integer.BYTES);
+ values[i] = new byte[length];
+ byteBuffer.get(values[i]);
+ }
+ }
+}
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimizerCardinalityTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimizerCardinalityTest.java
new file mode 100644
index 000000000000..85f57beae7ac
--- /dev/null
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimizerCardinalityTest.java
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.segment.local.segment.creator;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentCreationDriverFactory;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.segment.spi.creator.SegmentVersion;
+import org.apache.pinot.spi.config.table.FieldConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+public class DictionaryOptimizerCardinalityTest {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(DictionaryOptimizerCardinalityTest.class);
+ private static final File INDEX_DIR = new File(DictionaryOptimizerCardinalityTest.class.toString());
+ private static File _segmentDirectory;
+ private static File _csvFile;
+
+ // Test cardinality based dictionary optimization for var-length data type columns
+ @Test
+ public void testDictionaryForMixedCardinalitiesStringType()
+ throws Exception {
+
+ ImmutableSegment heapSegment = ImmutableSegmentLoader.load(_segmentDirectory, ReadMode.heap);
+
+ Schema schema = heapSegment.getSegmentMetadata().getSchema();
+ for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+ // Skip virtual columns
+ if (fieldSpec.isVirtualColumn()) {
+ continue;
+ }
+
+ String columnName = fieldSpec.getName();
+ if ("low_cardinality_strings".equals(columnName)) {
+ Assert.assertTrue(heapSegment.getForwardIndex(columnName).isDictionaryEncoded(),
+ "Low cardinality columns should be dictionary encoded");
+ }
+
+ if ("high_cardinality_strings".equals(columnName)) {
+ Assert.assertFalse(heapSegment.getForwardIndex(columnName).isDictionaryEncoded(),
+ "High cardinality columns should be raw encoded");
+ }
+ }
+ }
+
+ @BeforeClass
+ private void setup()
+ throws Exception {
+
+ if (INDEX_DIR.exists()) {
+ FileUtils.deleteQuietly(INDEX_DIR);
+ }
+ INDEX_DIR.mkdirs();
+ _csvFile = new File(INDEX_DIR, "data.csv");
+ generateCsv(_csvFile, 500);
+
+ IngestionConfig ingestionConfig = new IngestionConfig();
+ ingestionConfig.setRowTimeValueCheck(false);
+ ingestionConfig.setSegmentTimeValueCheck(false);
+ Schema schema =
+ new Schema.SchemaBuilder().addSingleValueDimension("low_cardinality_strings", FieldSpec.DataType.STRING)
+ .addSingleValueDimension("high_cardinality_strings", FieldSpec.DataType.STRING)
+ .addDateTimeField("ts", FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS").build();
+ List stringColumns =
+ schema.getDimensionFieldSpecs().stream().filter(x -> x.getDataType() == FieldSpec.DataType.STRING).collect(
+ Collectors.toList());
+
+ List fieldConfigList = stringColumns.stream().map(
+ x -> new FieldConfig(x.getName(), FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null, null))
+ .collect(Collectors.toList());
+
+ SegmentGeneratorConfig segmentGenSpec =
+ new SegmentGeneratorConfig(new TableConfigBuilder(TableType.OFFLINE).setTableName("tableName")
+ .setIngestionConfig(ingestionConfig).setFieldConfigList(fieldConfigList).build(),
+ schema);
+
+ segmentGenSpec.setInputFilePath(_csvFile.getAbsolutePath());
+ segmentGenSpec.setTimeColumnName("ts");
+ segmentGenSpec.setSegmentTimeUnit(TimeUnit.SECONDS);
+ segmentGenSpec.setFormat(FileFormat.CSV);
+ segmentGenSpec.setSegmentVersion(SegmentVersion.v1);
+ segmentGenSpec.setTableName("tableName");
+ segmentGenSpec.setOutDir(INDEX_DIR.getAbsolutePath());
+ segmentGenSpec.setOptimizeDictionary(true);
+ segmentGenSpec.setNoDictionaryCardinalityRatioThreshold(0.1); // cardinality must be <10% of total docs to override
+
+ SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
+ driver.init(segmentGenSpec);
+ driver.build();
+
+ _segmentDirectory = new File(INDEX_DIR, driver.getSegmentName());
+ }
+
+ // Generate a 3 columns csv file, sample format is:
+ // low_cardinality_strings,high_cardinality_strings,ts
+ // Red,kdeejdfnsd,1600000000
+ private void generateCsv(File file, int numberOfRows) throws IOException {
+ String[] lowCardinalityOptions = {"Red", "Blue", "Green", "Yellow", "Purple"};
+ String alphabet = "abcdefghijklmnopqrstuvwxyz";
+ Random random = new Random(42);
+
+ try (FileWriter writer = new FileWriter(file, false)) {
+ // Write the header
+ writer.append("low_cardinality_strings,high_cardinality_strings,ts\n");
+
+ long startTimestamp = System.currentTimeMillis() / 1000;
+ for (int i = 0; i < numberOfRows; i++) {
+ String lowCardinality = lowCardinalityOptions[random.nextInt(lowCardinalityOptions.length)];
+ StringBuilder highCardinality = new StringBuilder(10);
+ for (int j = 0; j < 10; j++) {
+ highCardinality.append(alphabet.charAt(random.nextInt(alphabet.length())));
+ }
+ long timestamp = startTimestamp + (i / 10);
+ writer.append(String.format("%s,%s,%d\n", lowCardinality, highCardinality, timestamp));
+ }
+ }
+ }
+
+ @AfterClass
+ public void cleanup() {
+ FileUtils.deleteQuietly(_csvFile);
+ FileUtils.deleteQuietly(_segmentDirectory);
+ FileUtils.deleteQuietly(INDEX_DIR);
+ }
+}
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueFixedByteRawIndexCreatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueFixedByteRawIndexCreatorTest.java
index 33c8525a42d2..9a2105726aa7 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueFixedByteRawIndexCreatorTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueFixedByteRawIndexCreatorTest.java
@@ -49,10 +49,9 @@
public class MultiValueFixedByteRawIndexCreatorTest {
- private static final String OUTPUT_DIR =
- System.getProperty("java.io.tmpdir") + File.separator + "mvFixedRawTest";
+ protected static String _outputDir;
- private static final Random RANDOM = new Random();
+ protected static final Random RANDOM = new Random();
@DataProvider(name = "compressionTypes")
public Object[][] compressionTypes() {
@@ -64,7 +63,8 @@ public Object[][] compressionTypes() {
@BeforeClass
public void setup()
throws Exception {
- FileUtils.forceMkdir(new File(OUTPUT_DIR));
+ _outputDir = System.getProperty("java.io.tmpdir") + File.separator + "mvFixedRawTest";
+ FileUtils.forceMkdir(new File(_outputDir));
}
/**
@@ -72,7 +72,7 @@ public void setup()
*/
@AfterClass
public void cleanup() {
- FileUtils.deleteQuietly(new File(OUTPUT_DIR));
+ FileUtils.deleteQuietly(new File(_outputDir));
}
@Test(dataProvider = "compressionTypes")
@@ -147,25 +147,34 @@ public void testMVDouble(ChunkCompressionType compressionType, int writerVersion
}, compressionType, writerVersion);
}
+ public MultiValueFixedByteRawIndexCreator getMultiValueFixedByteRawIndexCreator(ChunkCompressionType compressionType,
+ String column, int numDocs, DataType dataType, int maxElements, int writerVersion)
+ throws IOException {
+ return new MultiValueFixedByteRawIndexCreator(new File(_outputDir), compressionType, column, numDocs, dataType,
+ maxElements, false, writerVersion, 1024 * 1024, 1000);
+ }
+
+ public ForwardIndexReader getForwardIndexReader(PinotDataBuffer buffer, DataType dataType, int writerVersion) {
+ return writerVersion == VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexReaderV4(buffer,
+ dataType.getStoredType(), false) : new FixedByteChunkMVForwardIndexReader(buffer, dataType.getStoredType());
+ }
+
public void testMV(DataType dataType, List inputs, ToIntFunction sizeof, IntFunction constructor,
Injector injector, Extractor extractor, ChunkCompressionType compressionType, int writerVersion)
throws IOException {
String column = "testCol_" + dataType;
int numDocs = inputs.size();
int maxElements = inputs.stream().mapToInt(sizeof).max().orElseThrow(RuntimeException::new);
- File file = new File(OUTPUT_DIR, column + Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION);
+ File file = new File(_outputDir, column + Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION);
file.delete();
MultiValueFixedByteRawIndexCreator creator =
- new MultiValueFixedByteRawIndexCreator(new File(OUTPUT_DIR), compressionType, column, numDocs, dataType,
- maxElements, false, writerVersion, 1024 * 1024, 1000);
+ getMultiValueFixedByteRawIndexCreator(compressionType, column, numDocs, dataType, maxElements, writerVersion);
inputs.forEach(input -> injector.inject(creator, input));
creator.close();
//read
final PinotDataBuffer buffer = PinotDataBuffer.mapFile(file, true, 0, file.length(), ByteOrder.BIG_ENDIAN, "");
- ForwardIndexReader reader =
- writerVersion == VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexReaderV4(buffer,
- dataType.getStoredType(), false) : new FixedByteChunkMVForwardIndexReader(buffer, dataType.getStoredType());
+ ForwardIndexReader reader = getForwardIndexReader(buffer, dataType, writerVersion);
final ForwardIndexReaderContext context = reader.createContext();
T valueBuffer = constructor.apply(maxElements);
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV4Test.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV4Test.java
index 70313d91e701..8387c93dc838 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV4Test.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV4Test.java
@@ -193,7 +193,7 @@ private void testWriteRead(File file, ChunkCompressionType compressionType,
}
}
- private Stream randomStrings(int count, int lengthOfLongestEntry) {
+ protected Stream randomStrings(int count, int lengthOfLongestEntry) {
return IntStream.range(0, count)
.mapToObj(i -> {
int length = ThreadLocalRandom.current().nextInt(lengthOfLongestEntry);
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV5Test.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV5Test.java
new file mode 100644
index 000000000000..c5bbc75c2760
--- /dev/null
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV5Test.java
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.segment.index.creator;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV5;
+import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueFixedByteRawIndexCreator;
+import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkForwardIndexReaderV5;
+import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
+import org.apache.pinot.segment.spi.memory.PinotDataBuffer;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+
+
+public class VarByteChunkV5Test extends VarByteChunkV4Test {
+ private static final Random RANDOM = new Random();
+ private static File[] _dirs;
+
+ @DataProvider(parallel = true)
+ public Object[][] params() {
+ Object[][] params = new Object[][]{
+ {null, ChunkCompressionType.LZ4, 20, 1024},
+ {null, ChunkCompressionType.LZ4_LENGTH_PREFIXED, 20, 1024},
+ {null, ChunkCompressionType.PASS_THROUGH, 20, 1024},
+ {null, ChunkCompressionType.SNAPPY, 20, 1024},
+ {null, ChunkCompressionType.ZSTANDARD, 20, 1024},
+ {null, ChunkCompressionType.LZ4, 2048, 1024},
+ {null, ChunkCompressionType.LZ4_LENGTH_PREFIXED, 2048, 1024},
+ {null, ChunkCompressionType.PASS_THROUGH, 2048, 1024},
+ {null, ChunkCompressionType.SNAPPY, 2048, 1024},
+ {null, ChunkCompressionType.ZSTANDARD, 2048, 1024}
+ };
+
+ for (int i = 0; i < _dirs.length; i++) {
+ params[i][0] = _dirs[i];
+ }
+
+ return params;
+ }
+
+ @BeforeClass
+ public void forceMkDirs()
+ throws IOException {
+ _dirs = new File[10];
+ for (int i = 0; i < _dirs.length; i++) {
+ _dirs[i] = new File(new File(FileUtils.getTempDirectory(), UUID.randomUUID().toString()), "VarByteChunkV5Test");
+ FileUtils.forceMkdir(_dirs[i]);
+ }
+ }
+
+ @AfterClass
+ public void deleteDirs() {
+ for (File dir : _dirs) {
+ FileUtils.deleteQuietly(dir);
+ }
+ }
+
+ @Test(dataProvider = "params")
+ public void testStringSV(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize)
+ throws IOException {
+ File stringSVFile = new File(file, "testStringSV");
+ testWriteRead(stringSVFile, compressionType, longestEntry, chunkSize, FieldSpec.DataType.STRING, x -> x,
+ VarByteChunkForwardIndexWriterV5::putString, (reader, context, docId) -> reader.getString(docId, context));
+ FileUtils.deleteQuietly(stringSVFile);
+ }
+
+ @Test(dataProvider = "params")
+ public void testBytesSV(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize)
+ throws IOException {
+ File bytesSVFile = new File(file, "testBytesSV");
+ testWriteRead(bytesSVFile, compressionType, longestEntry, chunkSize, FieldSpec.DataType.BYTES,
+ x -> x.getBytes(StandardCharsets.UTF_8), VarByteChunkForwardIndexWriterV5::putBytes,
+ (reader, context, docId) -> reader.getBytes(docId, context));
+ FileUtils.deleteQuietly(bytesSVFile);
+ }
+
+ @Test(dataProvider = "params")
+ public void testStringMV(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize)
+ throws IOException {
+ File stringMVFile = new File(file, "testStringMV");
+ testWriteRead(stringMVFile, compressionType, longestEntry, chunkSize, FieldSpec.DataType.STRING,
+ new StringSplitterMV(), VarByteChunkForwardIndexWriterV5::putStringMV,
+ (reader, context, docId) -> reader.getStringMV(docId, context));
+ FileUtils.deleteQuietly(stringMVFile);
+ }
+
+ @Test(dataProvider = "params")
+ public void testBytesMV(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize)
+ throws IOException {
+ File bytesMVFile = new File(file, "testBytesMV");
+ testWriteRead(bytesMVFile, compressionType, longestEntry, chunkSize, FieldSpec.DataType.BYTES, new ByteSplitterMV(),
+ VarByteChunkForwardIndexWriterV5::putBytesMV, (reader, context, docId) -> reader.getBytesMV(docId, context));
+ FileUtils.deleteQuietly(bytesMVFile);
+ }
+
+ @Test
+ public void validateCompressionRatioIncrease()
+ throws IOException {
+ // Generate input data containing short MV docs with somewhat repetitive data
+ int numDocs = 1000000;
+ int numElements = 0;
+ int maxMVRowSize = 0;
+ List inputData = new ArrayList<>(numDocs);
+ for (int i = 0; i < numDocs; i++) {
+ long[] mvRow = new long[Math.abs((int) Math.floor(RANDOM.nextGaussian()))];
+ maxMVRowSize = Math.max(maxMVRowSize, mvRow.length);
+ numElements += mvRow.length;
+ for (int j = 0; j < mvRow.length; j++, numElements++) {
+ mvRow[j] = numElements % 10;
+ }
+ inputData.add(mvRow);
+ }
+
+ for (int i = 0; i < _dirs.length; i++) {
+ _dirs[i] = new File(new File(FileUtils.getTempDirectory(), UUID.randomUUID().toString()), "VarByteChunkV5Test");
+ FileUtils.forceMkdir(_dirs[i]);
+ }
+
+ // Generate MV fixed byte raw fwd index with explicit length
+ int rawIndexVersionV4 = 4;
+ File explicitLengthFwdIndexFile = new File(FileUtils.getTempDirectory(), Integer.toString(rawIndexVersionV4));
+ FileUtils.deleteQuietly(explicitLengthFwdIndexFile);
+ try (MultiValueFixedByteRawIndexCreator creator = new MultiValueFixedByteRawIndexCreator(explicitLengthFwdIndexFile,
+ ChunkCompressionType.ZSTANDARD, numDocs, FieldSpec.DataType.LONG, numElements, true, rawIndexVersionV4)) {
+ for (long[] mvRow : inputData) {
+ creator.putLongMV(mvRow);
+ }
+ }
+
+ // Generate MV fixed byte raw fwd index with implicit length
+ int rawIndexVersionV5 = 5;
+ File implicitLengthFwdIndexFile = new File(FileUtils.getTempDirectory(), Integer.toString(rawIndexVersionV5));
+ FileUtils.deleteQuietly(implicitLengthFwdIndexFile);
+ try (MultiValueFixedByteRawIndexCreator creator = new MultiValueFixedByteRawIndexCreator(implicitLengthFwdIndexFile,
+ ChunkCompressionType.ZSTANDARD, numDocs, FieldSpec.DataType.LONG, numElements, true, rawIndexVersionV5)) {
+ for (long[] mvRow : inputData) {
+ creator.putLongMV(mvRow);
+ }
+ }
+
+ // For the input data, the explicit length compressed MV fixed byte raw forward index is expected to be at least
+ // 2x larger size in explicit length variant in V4 compared to the new implicit length variant in V5
+ long expectedImplicitLengthFwdIndexMaxSize = Math.round(implicitLengthFwdIndexFile.length() * 2.0d);
+ Assert.assertTrue(expectedImplicitLengthFwdIndexMaxSize < explicitLengthFwdIndexFile.length());
+
+ // Cleanup
+ FileUtils.deleteQuietly(explicitLengthFwdIndexFile);
+ FileUtils.deleteQuietly(implicitLengthFwdIndexFile);
+ }
+
+ private void testWriteRead(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize,
+ FieldSpec.DataType dataType, Function forwardMapper,
+ BiConsumer write, Read read)
+ throws IOException {
+ List values = randomStrings(1000, longestEntry).map(forwardMapper).collect(Collectors.toList());
+ try (VarByteChunkForwardIndexWriterV5 writer = new VarByteChunkForwardIndexWriterV5(file, compressionType,
+ chunkSize)) {
+ for (T value : values) {
+ write.accept(writer, value);
+ }
+ }
+ try (PinotDataBuffer buffer = PinotDataBuffer.mapReadOnlyBigEndianFile(file)) {
+ try (VarByteChunkForwardIndexReaderV5 reader = new VarByteChunkForwardIndexReaderV5(buffer, dataType, true);
+ VarByteChunkForwardIndexReaderV5.ReaderContext context = reader.createContext()) {
+ for (int i = 0; i < values.size(); i++) {
+ assertEquals(read.read(reader, context, i), values.get(i));
+ }
+ for (int i = 0; i < values.size(); i += 2) {
+ assertEquals(read.read(reader, context, i), values.get(i));
+ }
+ for (int i = 1; i < values.size(); i += 2) {
+ assertEquals(read.read(reader, context, i), values.get(i));
+ }
+ for (int i = 1; i < values.size(); i += 100) {
+ assertEquals(read.read(reader, context, i), values.get(i));
+ }
+ for (int i = values.size() - 1; i >= 0; i--) {
+ assertEquals(read.read(reader, context, i), values.get(i));
+ }
+ for (int i = values.size() - 1; i >= 0; i -= 2) {
+ assertEquals(read.read(reader, context, i), values.get(i));
+ }
+ for (int i = values.size() - 2; i >= 0; i -= 2) {
+ assertEquals(read.read(reader, context, i), values.get(i));
+ }
+ for (int i = values.size() - 1; i >= 0; i -= 100) {
+ assertEquals(read.read(reader, context, i), values.get(i));
+ }
+ }
+ }
+ }
+
+ @FunctionalInterface
+ interface Read {
+ T read(VarByteChunkForwardIndexReaderV5 reader, VarByteChunkForwardIndexReaderV5.ReaderContext context, int docId);
+ }
+}
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexTypeTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexTypeTest.java
index ec82432ab14d..83af69e260d5 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexTypeTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexTypeTest.java
@@ -27,15 +27,16 @@
import org.apache.pinot.segment.local.segment.index.AbstractSerdeIndexContract;
import org.apache.pinot.segment.spi.index.DictionaryIndexConfig;
import org.apache.pinot.segment.spi.index.FieldIndexConfigs;
+import org.apache.pinot.segment.spi.index.IndexType;
import org.apache.pinot.segment.spi.index.StandardIndexes;
import org.apache.pinot.spi.config.table.FieldConfig;
import org.apache.pinot.spi.config.table.IndexConfig;
import org.apache.pinot.spi.config.table.Intern;
-import org.apache.pinot.spi.config.table.JsonIndexConfig;
import org.apache.pinot.spi.data.DimensionFieldSpec;
import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.MetricFieldSpec;
import org.apache.pinot.spi.utils.JsonUtils;
+import org.mockito.Mockito;
import org.testng.Assert;
import org.testng.annotations.Test;
@@ -349,28 +350,60 @@ public void testStandardIndex() {
+ "the DictionaryIndexType static instance");
}
+ /**
+ * Tests to verify various combinations of inputs to test dictionary override optimization.
+ */
@Test
public void testDictionaryOverride() {
MetricFieldSpec metric = new MetricFieldSpec("testCol", FieldSpec.DataType.DOUBLE);
- FieldIndexConfigs fieldIndexConfigs = new FieldIndexConfigs.Builder().build();
+ IndexType index1 = Mockito.mock(IndexType.class);
+ Mockito.when(index1.getId()).thenReturn("index1");
+ IndexConfig indexConf = new IndexConfig(true);
+ FieldIndexConfigs fieldIndexConfigs = new FieldIndexConfigs.Builder().add(index1, indexConf).build();
// No need to disable dictionary
- assertTrue(DictionaryIndexType.ignoreDictionaryOverride(false, true, 2, metric, fieldIndexConfigs, 5, 20));
+ boolean result =
+ DictionaryIndexType.ignoreDictionaryOverride(false, true, 2, null, metric, fieldIndexConfigs, 5, 20);
+ assertTrue(result);
// Set a higher noDictionarySizeRatioThreshold
- assertFalse(DictionaryIndexType.ignoreDictionaryOverride(false, true, 5, metric, fieldIndexConfigs, 5, 20));
+ result = DictionaryIndexType.ignoreDictionaryOverride(false, true, 5, null, metric, fieldIndexConfigs, 5, 20);
+ assertFalse(result);
// optimizeDictionary and optimizeDictionaryForMetrics both turned on
- assertFalse(DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, metric, fieldIndexConfigs, 5, 20));
+ result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, null, metric, fieldIndexConfigs, 5, 20);
+ assertFalse(result);
+
+ // noDictionarySizeRatioThreshold and noDictionaryCardinalityThreshold are provided
+ result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, 0.10, metric, fieldIndexConfigs, 5, 100);
+ assertTrue(result);
+
+ // cardinality is much less than total docs, use dictionary
+ metric.setDataType(FieldSpec.DataType.STRING);
+ result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, 0.10, metric, fieldIndexConfigs, 5, 100);
+ assertTrue(result);
+
+ // cardinality is large % of total docs, do not use dictionary
+ result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, 0.10, metric, fieldIndexConfigs, 5, 20);
+ assertFalse(result);
+
+ // Test Dimension col
+ // Don't ignore for Json. We want to disable dictionary for json.
+ DimensionFieldSpec dimension = new DimensionFieldSpec("testCol", FieldSpec.DataType.JSON, true);
+ result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, null, dimension, fieldIndexConfigs, 5, 20);
+ assertTrue(result);
+
+ // cardinality is much less than total docs, use dictionary
+ dimension.setDataType(FieldSpec.DataType.STRING);
+ result = DictionaryIndexType.ignoreDictionaryOverride(true, false, 5, 0.10, dimension, fieldIndexConfigs, 5, 100);
+ assertTrue(result);
+
+ // cardinality is large % of total docs, do not use dictionary
+ result = DictionaryIndexType.ignoreDictionaryOverride(true, false, 5, 0.10, dimension, fieldIndexConfigs, 5, 20);
+ assertFalse(result);
// Ignore for inverted index
IndexConfig indexConfig = new IndexConfig(false);
fieldIndexConfigs = new FieldIndexConfigs.Builder().add(StandardIndexes.inverted(), indexConfig).build();
- assertTrue(DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, metric, fieldIndexConfigs, 5, 20));
-
- // Don't ignore for JSON index
- DimensionFieldSpec dimension = new DimensionFieldSpec("testCol", FieldSpec.DataType.JSON, true);
- JsonIndexConfig jsonIndexConfig = new JsonIndexConfig();
- fieldIndexConfigs = new FieldIndexConfigs.Builder().add(StandardIndexes.json(), jsonIndexConfig).build();
- assertFalse(DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, dimension, fieldIndexConfigs, 5, 20));
+ assertTrue(DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, null, metric, fieldIndexConfigs, 5, 20));
}
}
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
index 5488b2cf2790..55b8227d41af 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
@@ -783,7 +783,7 @@ private void validateIndex(IndexType, ?, ?> indexType, String column, int card
assertFalse(reader.hasIndexFor(column, StandardIndexes.inverted()));
} else {
// Updating dictionary or forward index for existing columns not supported for v1 segments yet
- if (segmentMetadata.getVersion() == SegmentVersion.v3) {
+ if (segmentMetadata.getVersion() == SegmentVersion.v3 || isAutoGenerated) {
assertFalse(reader.hasIndexFor(column, StandardIndexes.forward()));
} else {
assertTrue(reader.hasIndexFor(column, StandardIndexes.forward()));
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java
deleted file mode 100644
index 47ea357894c8..000000000000
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pinot.segment.local.segment.index.loader.defaultcolumn;
-
-import com.google.common.collect.ImmutableMap;
-import java.io.File;
-import java.nio.file.Files;
-import java.util.Collections;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.io.FileUtils;
-import org.apache.pinot.segment.local.segment.creator.SegmentTestUtils;
-import org.apache.pinot.segment.local.segment.creator.impl.SegmentCreationDriverFactory;
-import org.apache.pinot.segment.local.segment.index.SegmentMetadataImplTest;
-import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
-import org.apache.pinot.segment.local.segment.store.SegmentLocalFSDirectory;
-import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
-import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
-import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl;
-import org.apache.pinot.segment.spi.store.SegmentDirectory;
-import org.apache.pinot.spi.data.FieldSpec;
-import org.apache.pinot.spi.data.Schema;
-import org.apache.pinot.spi.utils.ReadMode;
-import org.apache.pinot.util.TestUtils;
-import org.testng.Assert;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-
-public class BaseDefaultColumnHandlerTest {
- private static final String AVRO_DATA = "data/test_data-mv.avro";
-
- private File _indexDir;
- private File _segmentDirectory;
- private SegmentMetadataImpl _committedSegmentMetadata;
- private SegmentDirectory.Writer _writer;
-
- @BeforeMethod
- public void setUp()
- throws Exception {
- _indexDir = Files.createTempDirectory(SegmentMetadataImplTest.class.getName() + "_segmentDir").toFile();
-
- final String filePath =
- TestUtils.getFileFromResourceUrl(SegmentMetadataImplTest.class.getClassLoader().getResource(AVRO_DATA));
-
- // intentionally changed this to TimeUnit.Hours to make it non-default for testing
- final SegmentGeneratorConfig config = SegmentTestUtils
- .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), _indexDir, "daysSinceEpoch", TimeUnit.HOURS,
- "testTable");
- config.setSegmentNamePostfix("1");
- config.setTimeColumnName("daysSinceEpoch");
- final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
- driver.init(config);
- driver.build();
- _segmentDirectory = new File(_indexDir, driver.getSegmentName());
- _committedSegmentMetadata = new SegmentMetadataImpl(_segmentDirectory);
- _writer = new SegmentLocalFSDirectory(_segmentDirectory, _committedSegmentMetadata, ReadMode.mmap).createWriter();
- }
-
- @AfterMethod
- public void tearDown() {
- FileUtils.deleteQuietly(_segmentDirectory);
- }
-
- @Test
- public void testComputeDefaultColumnActionMapForCommittedSegment() {
- // Dummy IndexLoadingConfig
- IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
-
- // Same schema
- Schema schema0 =
- new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT)
- .addSingleValueDimension("column2", FieldSpec.DataType.INT)
- .addSingleValueDimension("column3", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column4", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column5", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column6", FieldSpec.DataType.INT)
- .addSingleValueDimension("column7", FieldSpec.DataType.INT)
- .addSingleValueDimension("column8", FieldSpec.DataType.INT)
- .addSingleValueDimension("column9", FieldSpec.DataType.INT)
- .addSingleValueDimension("column10", FieldSpec.DataType.INT)
- .addSingleValueDimension("column13", FieldSpec.DataType.INT)
- .addSingleValueDimension("count", FieldSpec.DataType.INT)
- .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT)
- .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build();
-
- BaseDefaultColumnHandler defaultColumnHandler =
- new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema0, _writer);
- Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), Collections.EMPTY_MAP);
-
- // Add single-value dimension in the schema
- Schema schema1 =
- new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT)
- .addSingleValueDimension("column2", FieldSpec.DataType.INT)
- .addSingleValueDimension("column3", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column4", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column5", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column6", FieldSpec.DataType.INT)
- .addSingleValueDimension("column7", FieldSpec.DataType.INT)
- .addSingleValueDimension("column8", FieldSpec.DataType.INT)
- .addSingleValueDimension("column9", FieldSpec.DataType.INT)
- .addSingleValueDimension("column10", FieldSpec.DataType.INT)
- .addSingleValueDimension("column11", FieldSpec.DataType.INT) // add column11
- .addSingleValueDimension("column13", FieldSpec.DataType.INT)
- .addSingleValueDimension("count", FieldSpec.DataType.INT)
- .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT)
- .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build();
- defaultColumnHandler =
- new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema1, _writer);
- Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(),
- ImmutableMap.of("column11", BaseDefaultColumnHandler.DefaultColumnAction.ADD_DIMENSION));
-
- // Add multi-value dimension in the schema
- Schema schema2 =
- new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT)
- .addSingleValueDimension("column2", FieldSpec.DataType.INT)
- .addSingleValueDimension("column3", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column4", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column5", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column6", FieldSpec.DataType.INT)
- .addSingleValueDimension("column7", FieldSpec.DataType.INT)
- .addSingleValueDimension("column8", FieldSpec.DataType.INT)
- .addSingleValueDimension("column9", FieldSpec.DataType.INT)
- .addSingleValueDimension("column10", FieldSpec.DataType.INT)
- .addMultiValueDimension("column11", FieldSpec.DataType.INT) // add column11
- .addSingleValueDimension("column13", FieldSpec.DataType.INT)
- .addSingleValueDimension("count", FieldSpec.DataType.INT)
- .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT)
- .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build();
- defaultColumnHandler =
- new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema2, _writer);
- Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(),
- ImmutableMap.of("column11", BaseDefaultColumnHandler.DefaultColumnAction.ADD_DIMENSION));
-
- // Add metric in the schema
- Schema schema3 =
- new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT)
- .addSingleValueDimension("column2", FieldSpec.DataType.INT)
- .addSingleValueDimension("column3", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column4", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column5", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column6", FieldSpec.DataType.INT)
- .addSingleValueDimension("column7", FieldSpec.DataType.INT)
- .addSingleValueDimension("column8", FieldSpec.DataType.INT)
- .addSingleValueDimension("column9", FieldSpec.DataType.INT)
- .addSingleValueDimension("column10", FieldSpec.DataType.INT)
- .addSingleValueDimension("column13", FieldSpec.DataType.INT)
- .addSingleValueDimension("count", FieldSpec.DataType.INT)
- .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT)
- .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT)
- .addMetric("column11", FieldSpec.DataType.INT).build(); // add column11
- defaultColumnHandler =
- new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema3, _writer);
- Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(),
- ImmutableMap.of("column11", BaseDefaultColumnHandler.DefaultColumnAction.ADD_METRIC));
-
- // Add metric in the schema
- Schema schema4 =
- new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT)
- .addSingleValueDimension("column2", FieldSpec.DataType.INT)
- .addSingleValueDimension("column3", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column4", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column5", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column6", FieldSpec.DataType.INT)
- .addSingleValueDimension("column7", FieldSpec.DataType.INT)
- .addSingleValueDimension("column8", FieldSpec.DataType.INT)
- .addSingleValueDimension("column9", FieldSpec.DataType.INT)
- .addSingleValueDimension("column10", FieldSpec.DataType.INT)
- .addSingleValueDimension("column13", FieldSpec.DataType.INT)
- .addSingleValueDimension("count", FieldSpec.DataType.INT)
- .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT)
- .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT)
- .addDateTime("column11", FieldSpec.DataType.INT, "1:HOURS:EPOCH", "1:HOURS").build(); // add column11
- defaultColumnHandler =
- new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema4, _writer);
- Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(),
- ImmutableMap.of("column11", BaseDefaultColumnHandler.DefaultColumnAction.ADD_DATE_TIME));
-
- // Do not remove non-autogenerated column in the segmentMetadata
- Schema schema5 =
- new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT)
- .addSingleValueDimension("column3", FieldSpec.DataType.STRING) // remove column2
- .addSingleValueDimension("column4", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column5", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column6", FieldSpec.DataType.INT)
- .addSingleValueDimension("column7", FieldSpec.DataType.INT)
- .addSingleValueDimension("column8", FieldSpec.DataType.INT)
- .addSingleValueDimension("column9", FieldSpec.DataType.INT)
- .addSingleValueDimension("column10", FieldSpec.DataType.INT)
- .addSingleValueDimension("column13", FieldSpec.DataType.INT)
- .addSingleValueDimension("count", FieldSpec.DataType.INT)
- .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT)
- .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build();
- defaultColumnHandler =
- new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema5, _writer);
- Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), Collections.EMPTY_MAP);
-
- // Do not update non-autogenerated column in the schema
- Schema schema6 =
- new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT)
- .addSingleValueDimension("column2", FieldSpec.DataType.STRING) // update datatype
- .addSingleValueDimension("column3", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column4", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column5", FieldSpec.DataType.STRING)
- .addSingleValueDimension("column6", FieldSpec.DataType.INT)
- .addSingleValueDimension("column7", FieldSpec.DataType.INT)
- .addSingleValueDimension("column8", FieldSpec.DataType.INT)
- .addSingleValueDimension("column9", FieldSpec.DataType.INT)
- .addSingleValueDimension("column10", FieldSpec.DataType.INT)
- .addSingleValueDimension("column13", FieldSpec.DataType.INT)
- .addSingleValueDimension("count", FieldSpec.DataType.INT)
- .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT)
- .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build();
- defaultColumnHandler =
- new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema6, _writer);
- Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), Collections.EMPTY_MAP);
- }
-}
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnHandlerTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnHandlerTest.java
new file mode 100644
index 000000000000..dc008a487dfe
--- /dev/null
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnHandlerTest.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.segment.index.loader.defaultcolumn;
+
+import java.io.File;
+import java.net.URL;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.creator.SegmentTestUtils;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.local.segment.index.loader.defaultcolumn.BaseDefaultColumnHandler.DefaultColumnAction;
+import org.apache.pinot.segment.local.segment.store.SegmentLocalFSDirectory;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.DateTimeFieldSpec;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.data.MetricFieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+
+
+public class DefaultColumnHandlerTest {
+ private static final String RAW_TABLE_NAME = "testTable";
+ private static final String SEGMENT_NAME = "testSegment";
+ private static final File TEMP_DIR =
+ new File(FileUtils.getTempDirectory(), DefaultColumnHandlerTest.class.getSimpleName());
+ private static final File INDEX_DIR = new File(TEMP_DIR, SEGMENT_NAME);
+ private static final String AVRO_DATA = "data/test_data-mv.avro";
+
+ private static final TableConfig TABLE_CONFIG =
+ new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+ private Schema _schema;
+ private SegmentDirectory _segmentDirectory;
+ private SegmentDirectory.Writer _writer;
+
+ @BeforeClass
+ public void setUp()
+ throws Exception {
+ FileUtils.deleteQuietly(TEMP_DIR);
+
+ URL resourceUrl = getClass().getClassLoader().getResource(AVRO_DATA);
+ assertNotNull(resourceUrl);
+ File avroFile = new File(resourceUrl.getFile());
+ _schema = SegmentTestUtils.extractSchemaFromAvroWithoutTime(avroFile);
+ SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, _schema);
+ config.setInputFilePath(avroFile.getAbsolutePath());
+ config.setOutDir(TEMP_DIR.getAbsolutePath());
+ config.setSegmentName(SEGMENT_NAME);
+ SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+ driver.init(config);
+ driver.build();
+ }
+
+ @AfterClass
+ public void tearDown() {
+ FileUtils.deleteQuietly(TEMP_DIR);
+ }
+
+ @Test
+ public void testComputeDefaultColumnActionMap()
+ throws Exception {
+ try (SegmentDirectory segmentDirectory = new SegmentLocalFSDirectory(INDEX_DIR, ReadMode.mmap);
+ SegmentDirectory.Writer writer = segmentDirectory.createWriter()) {
+ _segmentDirectory = segmentDirectory;
+ _writer = writer;
+
+ // Same schema
+ testComputeDefaultColumnActionMap(Map.of());
+
+ // Add single-value dimension in the schema
+ _schema.addField(new DimensionFieldSpec("column11", DataType.INT, true));
+ testComputeDefaultColumnActionMap(Map.of("column11", DefaultColumnAction.ADD_DIMENSION));
+ _schema.removeField("column11");
+
+ // Add multi-value dimension in the schema
+ _schema.addField(new DimensionFieldSpec("column11", DataType.INT, false));
+ testComputeDefaultColumnActionMap(Map.of("column11", DefaultColumnAction.ADD_DIMENSION));
+ _schema.removeField("column11");
+
+ // Add metric in the schema
+ _schema.addField(new MetricFieldSpec("column11", DataType.INT));
+ testComputeDefaultColumnActionMap(Map.of("column11", DefaultColumnAction.ADD_METRIC));
+ _schema.removeField("column11");
+
+ // Add date-time in the schema
+ _schema.addField(new DateTimeFieldSpec("column11", DataType.INT, "EPOCH|HOURS", "1:HOURS"));
+ testComputeDefaultColumnActionMap(Map.of("column11", DefaultColumnAction.ADD_DATE_TIME));
+ _schema.removeField("column11");
+
+ // Do not remove non-autogenerated column in the segmentMetadata
+ _schema.removeField("column2");
+ testComputeDefaultColumnActionMap(Map.of());
+
+ // Do not update non-autogenerated column in the schema
+ _schema.addField(new DimensionFieldSpec("column2", DataType.STRING, true));
+ testComputeDefaultColumnActionMap(Map.of());
+ }
+ }
+
+ private void testComputeDefaultColumnActionMap(Map expected) {
+ BaseDefaultColumnHandler defaultColumnHandler =
+ new V3DefaultColumnHandler(INDEX_DIR, _segmentDirectory.getSegmentMetadata(),
+ new IndexLoadingConfig(TABLE_CONFIG, _schema), _schema, _writer);
+ assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), expected);
+ }
+}
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java
index 4424be0883e6..f5065d417f4a 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java
@@ -127,6 +127,7 @@ public enum TimeColumnType {
private boolean _optimizeDictionary = false;
private boolean _optimizeDictionaryForMetrics = false;
private double _noDictionarySizeRatioThreshold = IndexingConfig.DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD;
+ private Double _noDictionaryCardinalityRatioThreshold;
private boolean _realtimeConversion = false;
// consumerDir contains data from the consuming segment, and is used during _realtimeConversion optimization
private File _consumerDir;
@@ -208,6 +209,7 @@ public SegmentGeneratorConfig(TableConfig tableConfig, Schema schema) {
_optimizeDictionary = indexingConfig.isOptimizeDictionary();
_optimizeDictionaryForMetrics = indexingConfig.isOptimizeDictionaryForMetrics();
_noDictionarySizeRatioThreshold = indexingConfig.getNoDictionarySizeRatioThreshold();
+ _noDictionaryCardinalityRatioThreshold = indexingConfig.getNoDictionaryCardinalityRatioThreshold();
}
IngestionConfig ingestionConfig = tableConfig.getIngestionConfig();
@@ -805,6 +807,16 @@ public void setNoDictionarySizeRatioThreshold(double noDictionarySizeRatioThresh
_noDictionarySizeRatioThreshold = noDictionarySizeRatioThreshold;
}
+ @Nullable
+ public Double getNoDictionaryCardinalityRatioThreshold() {
+ return _noDictionaryCardinalityRatioThreshold;
+ }
+
+ public void setNoDictionaryCardinalityRatioThreshold(@Nullable Double noDictionaryCardinalityRatioThreshold) {
+ _noDictionaryCardinalityRatioThreshold = noDictionaryCardinalityRatioThreshold;
+ }
+
+
public boolean isFailOnEmptySegment() {
return _failOnEmptySegment;
}
diff --git a/pinot-spi/pom.xml b/pinot-spi/pom.xml
index 5645b5de63ce..ec0016243112 100644
--- a/pinot-spi/pom.xml
+++ b/pinot-spi/pom.xml
@@ -238,5 +238,11 @@
package
+
+ pinot-fastdev
+
+ none
+
+
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
index 1cfd6107107c..369f27767507 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
@@ -27,7 +27,7 @@
public class IndexingConfig extends BaseJsonConfig {
- // Default ratio for overriding dictionary
+ // Default ratio for overriding dictionary for fixed width columns
public static final double DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD = 0.85d;
/**
@@ -78,6 +78,10 @@ public class IndexingConfig extends BaseJsonConfig {
private double _noDictionarySizeRatioThreshold = DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD;
+ // Used in conjunction with `optimizeDictionary`, if cardinality / total docs is less than the threshold,
+ // then create a dictionary for the column. A value around 0.1 (10%) is a reasonable starting point
+ private Double _noDictionaryCardinalityRatioThreshold;
+
// TODO: Add a new configuration related to the segment generation
private boolean _autoGeneratedInvertedIndex;
private boolean _createInvertedIndexDuringSegmentGeneration;
@@ -372,6 +376,15 @@ public void setNoDictionarySizeRatioThreshold(double noDictionarySizeRatioThresh
_noDictionarySizeRatioThreshold = noDictionarySizeRatioThreshold;
}
+ @Nullable
+ public Double getNoDictionaryCardinalityRatioThreshold() {
+ return _noDictionaryCardinalityRatioThreshold;
+ }
+
+ public void setNoDictionaryCardinalityRatioThreshold(Double noDictionaryCardinalityRatioThreshold) {
+ _noDictionaryCardinalityRatioThreshold = noDictionaryCardinalityRatioThreshold;
+ }
+
public String getSegmentNameGeneratorType() {
return _segmentNameGeneratorType;
}
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
index 56c03698f1a8..ff81f6bc4ea8 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
@@ -343,7 +343,11 @@ public static class Broker {
public static final String CONFIG_OF_ENABLE_PARTITION_METADATA_MANAGER =
"pinot.broker.enable.partition.metadata.manager";
- public static final boolean DEFAULT_ENABLE_PARTITION_METADATA_MANAGER = false;
+ public static final boolean DEFAULT_ENABLE_PARTITION_METADATA_MANAGER = true;
+ // Whether to infer partition hint by default or not.
+ // This value can always be overridden by INFER_PARTITION_HINT query option
+ public static final String CONFIG_OF_INFER_PARTITION_HINT = "pinot.broker.multistage.infer.partition.hint";
+ public static final boolean DEFAULT_INFER_PARTITION_HINT = false;
public static final String CONFIG_OF_USE_FIXED_REPLICA = "pinot.broker.use.fixed.replica";
public static final boolean DEFAULT_USE_FIXED_REPLICA = false;
@@ -393,6 +397,7 @@ public static class QueryOptionKey {
public static final String USE_FIXED_REPLICA = "useFixedReplica";
public static final String EXPLAIN_PLAN_VERBOSE = "explainPlanVerbose";
public static final String USE_MULTISTAGE_ENGINE = "useMultistageEngine";
+ public static final String INFER_PARTITION_HINT = "inferPartitionHint";
public static final String ENABLE_NULL_HANDLING = "enableNullHandling";
/**
* If set, changes the explain behavior in multi-stage engine.
@@ -458,6 +463,15 @@ public static class QueryOptionKey {
// executed in an Unbounded FCFS fashion. However, secondary workloads are executed in a constrainted FCFS
// fashion with limited compute.
public static final String IS_SECONDARY_WORKLOAD = "isSecondaryWorkload";
+
+ // For group by queries with only filtered aggregations (and no non-filtered aggregations), the default behavior
+ // is to compute all groups over the rows matching the main query filter. This ensures SQL compliant results,
+ // since empty groups are also expected to be returned in such queries. However, this could be quite inefficient
+ // if the main query does not have a filter (since a scan would be required to compute all groups). In case
+ // users are okay with skipping empty groups - i.e., only the groups matching at least one aggregation filter
+ // will be returned - this query option can be set. This is useful for performance, since indexes can be used
+ // for the aggregation filters and a full scan can be avoided.
+ public static final String FILTERED_AGGREGATIONS_SKIP_EMPTY_GROUPS = "filteredAggregationsSkipEmptyGroups";
}
public static class QueryOptionValue {
diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/config/table/IndexingConfigTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/config/table/IndexingConfigTest.java
index f97e9b30b1a1..d3b9b8fc364a 100644
--- a/pinot-spi/src/test/java/org/apache/pinot/spi/config/table/IndexingConfigTest.java
+++ b/pinot-spi/src/test/java/org/apache/pinot/spi/config/table/IndexingConfigTest.java
@@ -18,6 +18,7 @@
*/
package org.apache.pinot.spi.config.table;
+import com.fasterxml.jackson.core.JsonProcessingException;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
@@ -27,6 +28,7 @@
import org.testng.annotations.Test;
import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
@@ -97,4 +99,27 @@ public void testSegmentPartitionConfig()
assertEquals(actualPartitionConfig.getNumPartitions(column), expectedPartitionConfig.getNumPartitions(column));
}
}
+
+ @Test
+ public void testOptimizeDictionaryConfigs()
+ throws JsonProcessingException {
+ String indexingConfigStr = "{"
+ + "\"optimizeDictionary\": true,"
+ + "\"optimizeDictionaryForMetrics\": true,"
+ + "\"noDictionarySizeRatioThreshold\": 0.50"
+ + "}";
+ IndexingConfig indexingConfig = JsonUtils.stringToObject(indexingConfigStr, IndexingConfig.class);
+ assertTrue(indexingConfig.isOptimizeDictionary());
+ assertTrue(indexingConfig.isOptimizeDictionaryForMetrics());
+ assertEquals(indexingConfig.getNoDictionarySizeRatioThreshold(), 0.50d);
+ assertNull(indexingConfig.getNoDictionaryCardinalityRatioThreshold());
+
+ indexingConfigStr = "{"
+ + "\"optimizeDictionary\": true,"
+ + "\"noDictionaryCardinalityRatioThreshold\": 0.07"
+ + "}";
+ indexingConfig = JsonUtils.stringToObject(indexingConfigStr, IndexingConfig.class);
+ assertTrue(indexingConfig.isOptimizeDictionary());
+ assertEquals(indexingConfig.getNoDictionaryCardinalityRatioThreshold(), 0.07d);
+ }
}
diff --git a/pinot-timeseries/pinot-timeseries-spi/src/main/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNode.java b/pinot-timeseries/pinot-timeseries-spi/src/main/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNode.java
index e56350845011..856b81622a3a 100644
--- a/pinot-timeseries/pinot-timeseries-spi/src/main/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNode.java
+++ b/pinot-timeseries/pinot-timeseries-spi/src/main/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNode.java
@@ -119,7 +119,7 @@ public String getEffectiveFilter(TimeBuckets timeBuckets) {
long endTime =
_timeUnit.convert(Duration.ofSeconds(
timeBuckets.getEndTime() + timeBuckets.getBucketSize().toSeconds() - _offsetSeconds));
- String addnFilter = String.format("%s >= %d AND %s <= %d", _timeColumn, startTime, _timeColumn, endTime);
+ String addnFilter = String.format("%s >= %d AND %s < %d", _timeColumn, startTime, _timeColumn, endTime);
if (filter.strip().isEmpty()) {
return addnFilter;
}
diff --git a/pinot-timeseries/pinot-timeseries-spi/src/test/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNodeTest.java b/pinot-timeseries/pinot-timeseries-spi/src/test/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNodeTest.java
index ece46c332a9a..f439bfc0285a 100644
--- a/pinot-timeseries/pinot-timeseries-spi/src/test/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNodeTest.java
+++ b/pinot-timeseries/pinot-timeseries-spi/src/test/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNodeTest.java
@@ -46,7 +46,7 @@ public void testGetEffectiveFilter() {
new LeafTimeSeriesPlanNode(ID, Collections.emptyList(), TABLE, TIME_COLUMN, TIME_UNIT, 0L, "", "value_col",
new AggInfo("SUM", null), Collections.singletonList("cityName"));
assertEquals(planNode.getEffectiveFilter(timeBuckets),
- "orderTime >= " + expectedStartTimeInFilter + " AND orderTime <= " + expectedEndTimeInFilter);
+ "orderTime >= " + expectedStartTimeInFilter + " AND orderTime < " + expectedEndTimeInFilter);
}
// Case-2: Offset, but empty filter
{
@@ -54,7 +54,7 @@ public void testGetEffectiveFilter() {
new LeafTimeSeriesPlanNode(ID, Collections.emptyList(), TABLE, TIME_COLUMN, TIME_UNIT, 123L, "", "value_col",
new AggInfo("SUM", null), Collections.singletonList("cityName"));
assertEquals(planNode.getEffectiveFilter(timeBuckets),
- "orderTime >= " + (expectedStartTimeInFilter - 123) + " AND orderTime <= " + (expectedEndTimeInFilter - 123));
+ "orderTime >= " + (expectedStartTimeInFilter - 123) + " AND orderTime < " + (expectedEndTimeInFilter - 123));
}
// Case-3: Offset and non-empty filter
{
@@ -62,7 +62,7 @@ public void testGetEffectiveFilter() {
new LeafTimeSeriesPlanNode(ID, Collections.emptyList(), TABLE, TIME_COLUMN, TIME_UNIT, 123L, nonEmptyFilter,
"value_col", new AggInfo("SUM", null), Collections.singletonList("cityName"));
assertEquals(planNode.getEffectiveFilter(timeBuckets),
- String.format("(%s) AND (orderTime >= %s AND orderTime <= %s)", nonEmptyFilter,
+ String.format("(%s) AND (orderTime >= %s AND orderTime < %s)", nonEmptyFilter,
(expectedStartTimeInFilter - 123), (expectedEndTimeInFilter - 123)));
}
// Case-4: Offset, and non-empty filter, and time-unit that is not seconds
@@ -71,7 +71,7 @@ public void testGetEffectiveFilter() {
new LeafTimeSeriesPlanNode(ID, Collections.emptyList(), TABLE, TIME_COLUMN, TimeUnit.MILLISECONDS, 123L,
nonEmptyFilter, "value_col", new AggInfo("SUM", null), Collections.singletonList("cityName"));
assertEquals(planNode.getEffectiveFilter(timeBuckets),
- String.format("(%s) AND (orderTime >= %s AND orderTime <= %s)", nonEmptyFilter,
+ String.format("(%s) AND (orderTime >= %s AND orderTime < %s)", nonEmptyFilter,
(expectedStartTimeInFilter * 1000 - 123 * 1000), (expectedEndTimeInFilter * 1000 - 123 * 1000)));
}
}
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/ColocatedJoinEngineQuickStart.java b/pinot-tools/src/main/java/org/apache/pinot/tools/ColocatedJoinEngineQuickStart.java
index c9d8b42e4561..7ef5260034b3 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/ColocatedJoinEngineQuickStart.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/ColocatedJoinEngineQuickStart.java
@@ -42,6 +42,7 @@ public List types() {
@Override
public Map getConfigOverrides() {
Map overrides = new HashMap<>(super.getConfigOverrides());
+ // This is actually not required anymore, but we are keeping it as reference
overrides.put(CommonConstants.Broker.CONFIG_OF_ENABLE_PARTITION_METADATA_MANAGER, "true");
return overrides;
}
diff --git a/pom.xml b/pom.xml
index 50f11f4c6ec2..102522577310 100644
--- a/pom.xml
+++ b/pom.xml
@@ -145,7 +145,7 @@
2.17.2
3.9.2
3.0.0
- 2.42
+ 2.45
2.6.1
1.6.14
5.17.14
@@ -173,7 +173,7 @@
4.1.1
1.37
0.15.0
- 0.4.5
+ 0.4.7
4.2.2
2.28.12
1.2.28
@@ -232,7 +232,7 @@
3.25.4
1.68.0
- 26.47.0
+ 26.49.0
1.1.1
1.7
2.33.0
@@ -245,24 +245,24 @@
2.1.0
- 3.27.0
+ 3.27.1
2.0.1
1.5.4
9.41.2
3.6.2
9.4.56.v20240826
7.0.0
- 5.7.0
+ 5.7.1
3.30.2-GA
1.78.1
0.27
5.15.0
- 2.2.16
+ 2.2.17
0.10.4
9.7.1
2.8
- 2.0.20
- 26.0.0
+ 2.0.21
+ 26.0.1
3.9.1
2.24.0
3.4
@@ -274,11 +274,11 @@
7.10.2
- 5.14.1
+ 5.14.2
3.17.1
1.20.2
2.3.232
- 3.1.19
+ 3.1.20
3.2.19
@@ -293,6 +293,7 @@
true
true
+ none
@@ -766,7 +767,7 @@
org.checkerframework
checker-qual
- 3.48.0
+ 3.48.1
org.codehaus.groovy
@@ -1390,7 +1391,7 @@
org.apache.datasketches
datasketches-java
- 6.0.0
+ 6.1.1
com.dynatrace.hash4j
@@ -2292,6 +2293,11 @@
auto-service
${google.auto-service.version}
+
+ org.immutables
+ value-processor
+ ${immutables.version}
+