Skip to content

Commit

Permalink
[feature](statistics)Support partition stats cache (#35517)
Browse files Browse the repository at this point in the history
Support cache partition level stats and show partition level cached
stats.
Nereids could call StatisticsCache.getPartitionStatistics to fetch
cached partition stats.

<!--Describe your changes.-->

## Further comments

If this is a relatively large or complex change, kick off the discussion
at [[email protected]](mailto:[email protected]) by explaining why
you chose the solution you did and what alternatives you considered,
etc...
  • Loading branch information
Jibing-Li authored and dataroaring committed May 31, 2024
1 parent cf82dc6 commit cc4a447
Show file tree
Hide file tree
Showing 14 changed files with 660 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@
import org.apache.doris.statistics.AnalysisManager;
import org.apache.doris.statistics.ColStatsMeta;
import org.apache.doris.statistics.ColumnStatistic;
import org.apache.doris.statistics.PartitionColumnStatistic;
import org.apache.doris.statistics.PartitionColumnStatisticCacheKey;
import org.apache.doris.statistics.ResultRow;
import org.apache.doris.statistics.TableStatsMeta;

Expand All @@ -44,6 +46,7 @@
import com.google.common.collect.Sets;

import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;

Expand Down Expand Up @@ -225,6 +228,32 @@ public ShowResultSet constructPartitionResultSet(List<ResultRow> resultRows, Tab
return new ShowResultSet(getMetaData(), result);
}

public ShowResultSet constructPartitionCachedColumnStats(
Map<PartitionColumnStatisticCacheKey, PartitionColumnStatistic> resultMap, TableIf tableIf) {
List<List<String>> result = Lists.newArrayList();
for (Map.Entry<PartitionColumnStatisticCacheKey, PartitionColumnStatistic> entry : resultMap.entrySet()) {
PartitionColumnStatisticCacheKey key = entry.getKey();
PartitionColumnStatistic value = entry.getValue();
List<String> row = Lists.newArrayList();
row.add(key.colName); // column_name
row.add(key.partId); // partition_name
long indexId = key.idxId;
String indexName = indexId == -1 ? tableIf.getName() : ((OlapTable) tableIf).getIndexNameById(indexId);
row.add(indexName); // index_name.
row.add(String.valueOf(value.count)); // count
row.add(String.valueOf(value.ndv.estimateCardinality())); // ndv
row.add(String.valueOf(value.numNulls)); // num_null
row.add(String.valueOf(value.minValue)); // min
row.add(String.valueOf(value.maxValue)); // max
row.add(String.valueOf(value.dataSize)); // data_size
row.add(value.updatedTime); // updated_time
row.add("N/A"); // update_rows
row.add("N/A"); // trigger
result.add(row);
}
return new ShowResultSet(getMetaData(), result);
}

public PartitionNames getPartitionNames() {
return partitionNames;
}
Expand Down
47 changes: 44 additions & 3 deletions fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,8 @@
import org.apache.doris.statistics.AutoAnalysisPendingJob;
import org.apache.doris.statistics.ColumnStatistic;
import org.apache.doris.statistics.Histogram;
import org.apache.doris.statistics.PartitionColumnStatistic;
import org.apache.doris.statistics.PartitionColumnStatisticCacheKey;
import org.apache.doris.statistics.ResultRow;
import org.apache.doris.statistics.StatisticsRepository;
import org.apache.doris.statistics.TableStatsMeta;
Expand Down Expand Up @@ -2636,9 +2638,14 @@ private void handleShowColumnStats() throws AnalysisException {
List<String> partNames = partitionNames.getPartitionNames() == null
? new ArrayList<>(tableIf.getPartitionNames())
: partitionNames.getPartitionNames();
List<ResultRow> partitionColumnStats =
StatisticsRepository.queryColumnStatisticsByPartitions(tableIf, columnNames, partNames);
resultSet = showColumnStatsStmt.constructPartitionResultSet(partitionColumnStats, tableIf);
if (showCache) {
resultSet = showColumnStatsStmt.constructPartitionCachedColumnStats(
getCachedPartitionColumnStats(columnNames, partNames, tableIf), tableIf);
} else {
List<ResultRow> partitionColumnStats =
StatisticsRepository.queryColumnStatisticsByPartitions(tableIf, columnNames, partNames);
resultSet = showColumnStatsStmt.constructPartitionResultSet(partitionColumnStats, tableIf);
}
} else {
if (isAllColumns && !showCache) {
getStatsForAllColumns(columnStatistics, tableIf);
Expand Down Expand Up @@ -2704,6 +2711,40 @@ private void getStatsForSpecifiedColumns(List<Pair<Pair<String, String>, ColumnS
}
}

private Map<PartitionColumnStatisticCacheKey, PartitionColumnStatistic> getCachedPartitionColumnStats(
Set<String> columnNames, List<String> partitionNames, TableIf tableIf) {
Map<PartitionColumnStatisticCacheKey, PartitionColumnStatistic> ret = new HashMap<>();
long catalogId = tableIf.getDatabase().getCatalog().getId();
long dbId = tableIf.getDatabase().getId();
long tableId = tableIf.getId();
for (String colName : columnNames) {
// Olap base index use -1 as index id.
List<Long> indexIds = Lists.newArrayList();
if (tableIf instanceof OlapTable) {
indexIds = ((OlapTable) tableIf).getMvColumnIndexIds(colName);
} else {
indexIds.add(-1L);
}
for (long indexId : indexIds) {
String indexName = tableIf.getName();
if (tableIf instanceof OlapTable) {
OlapTable olapTable = (OlapTable) tableIf;
indexName = olapTable.getIndexNameById(indexId == -1 ? olapTable.getBaseIndexId() : indexId);
}
if (indexName == null) {
continue;
}
for (String partName : partitionNames) {
PartitionColumnStatistic partitionStatistics = Env.getCurrentEnv().getStatisticsCache()
.getPartitionColumnStatistics(catalogId, dbId, tableId, indexId, partName, colName);
ret.put(new PartitionColumnStatisticCacheKey(catalogId, dbId, tableId, indexId, partName, colName),
partitionStatistics);
}
}
}
return ret;
}

public void handleShowColumnHist() {
// TODO: support histogram in the future.
ShowColumnHistStmt showColumnHistStmt = (ShowColumnHistStmt) stmt;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3177,7 +3177,8 @@ public TStatus updateStatsCache(TUpdateFollowerStatsCacheRequest request) throws
ColStatsData data = GsonUtils.GSON.fromJson(request.colStatsData, ColStatsData.class);
ColumnStatistic c = data.toColumnStatistic();
if (c == ColumnStatistic.UNKNOWN) {
Env.getCurrentEnv().getStatisticsCache().invalidate(k.catalogId, k.dbId, k.tableId, k.idxId, k.colName);
Env.getCurrentEnv().getStatisticsCache().invalidate(k.catalogId, k.dbId, k.tableId,
k.idxId, null, k.colName);
} else {
Env.getCurrentEnv().getStatisticsCache().updateColStatsCache(
k.catalogId, k.dbId, k.tableId, k.idxId, k.colName, c);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -704,7 +704,7 @@ public void invalidateLocalStats(long catalogId, long dbId, long tableId,
}
}
tableStats.removeColumn(indexName, column);
statisticsCache.invalidate(catalogId, dbId, tableId, indexId, column);
statisticsCache.invalidate(catalogId, dbId, tableId, indexId, null, column);
}
}
tableStats.updatedTime = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,7 @@
import org.apache.logging.log4j.Logger;
import org.json.JSONObject;

import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;

public class ColumnStatistic {
Expand Down Expand Up @@ -110,15 +108,14 @@ public ColumnStatistic(double count, double ndv, ColumnStatistic original, doubl
}

public static ColumnStatistic fromResultRow(List<ResultRow> resultRows) {
Map<String, ColumnStatistic> partitionIdToColStats = new HashMap<>();
ColumnStatistic columnStatistic = null;
try {
for (ResultRow resultRow : resultRows) {
String partId = resultRow.get(6);
if (partId == null) {
columnStatistic = fromResultRow(resultRow);
} else {
partitionIdToColStats.put(partId, fromResultRow(resultRow));
LOG.warn("Column statistics table shouldn't contain partition stats. [{}]", resultRow);
}
}
} catch (Throwable t) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,8 @@ protected Optional<ColumnStatistic> doLoad(StatisticsCacheKey key) {
columnStatistic = table.getColumnStatistic(key.colName);
} catch (Exception e) {
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("Exception to get column statistics by metadata."
+ "[Catalog:{}, DB:{}, Table:{}]",
key.catalogId, key.dbId, key.tableId), e);
LOG.debug("Exception to get column statistics by metadata. [Catalog:{}, DB:{}, Table:{}]",
key.catalogId, key.dbId, key.tableId, e);
}
}
}
Expand All @@ -69,7 +68,7 @@ protected Optional<ColumnStatistic> doLoad(StatisticsCacheKey key) {
}

private Optional<ColumnStatistic> loadFromStatsTable(StatisticsCacheKey key) {
List<ResultRow> columnResults = null;
List<ResultRow> columnResults;
try {
columnResults = StatisticsRepository.loadColStats(
key.catalogId, key.dbId, key.tableId, key.idxId, key.colName);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,168 @@
// 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.doris.statistics;

import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.catalog.Column;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.io.Hll;
import org.apache.doris.statistics.util.Hll128;
import org.apache.doris.statistics.util.StatisticsUtil;

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;

import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.util.Base64;
import java.util.List;
import java.util.StringJoiner;

public class PartitionColumnStatistic {

private static final Logger LOG = LogManager.getLogger(PartitionColumnStatistic.class);

public static PartitionColumnStatistic UNKNOWN = new PartitionColumnStatisticBuilder().setAvgSizeByte(1)
.setNdv(new Hll128()).setNumNulls(1).setCount(1).setMaxValue(Double.POSITIVE_INFINITY)
.setMinValue(Double.NEGATIVE_INFINITY)
.setIsUnknown(true).setUpdatedTime("")
.build();

public static PartitionColumnStatistic ZERO = new PartitionColumnStatisticBuilder().setAvgSizeByte(0)
.setNdv(new Hll128()).setNumNulls(0).setCount(0).setMaxValue(Double.NaN).setMinValue(Double.NaN)
.build();

public final double count;
public final Hll128 ndv;
public final double numNulls;
public final double dataSize;
public final double avgSizeByte;
public final double minValue;
public final double maxValue;
public final boolean isUnKnown;
public final LiteralExpr minExpr;
public final LiteralExpr maxExpr;
public final String updatedTime;

public PartitionColumnStatistic(double count, Hll128 ndv, double avgSizeByte,
double numNulls, double dataSize, double minValue, double maxValue,
LiteralExpr minExpr, LiteralExpr maxExpr, boolean isUnKnown,
String updatedTime) {
this.count = count;
this.ndv = ndv;
this.avgSizeByte = avgSizeByte;
this.numNulls = numNulls;
this.dataSize = dataSize;
this.minValue = minValue;
this.maxValue = maxValue;
this.minExpr = minExpr;
this.maxExpr = maxExpr;
this.isUnKnown = isUnKnown;
this.updatedTime = updatedTime;
}

public static PartitionColumnStatistic fromResultRow(List<ResultRow> resultRows) {
if (resultRows == null || resultRows.isEmpty()) {
return PartitionColumnStatistic.UNKNOWN;
}
// This should never happen. resultRows should be empty or contain only 1 result row.
if (resultRows.size() > 1) {
StringJoiner stringJoiner = new StringJoiner("][", "[", "]");
for (ResultRow row : resultRows) {
stringJoiner.add(row.toString());
}
LOG.warn("Partition stats has more than one row, please drop stats and analyze again. {}",
stringJoiner.toString());
return PartitionColumnStatistic.UNKNOWN;
}
try {
return fromResultRow(resultRows.get(0));
} catch (Throwable t) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to deserialize column stats", t);
}
return PartitionColumnStatistic.UNKNOWN;
}
}

public static PartitionColumnStatistic fromResultRow(ResultRow row) {
// row : [catalog_id, db_id, tbl_id, idx_id, col_id, count, ndv, null_count, min, max, data_size, update_time]
try {
long catalogId = Long.parseLong(row.get(0));
long dbID = Long.parseLong(row.get(1));
long tblId = Long.parseLong(row.get(2));
long idxId = Long.parseLong(row.get(3));
String colName = row.get(4);
Column col = StatisticsUtil.findColumn(catalogId, dbID, tblId, idxId, colName);
if (col == null) {
LOG.info("Failed to deserialize column statistics, ctlId: {} dbId: {}, "
+ "tblId: {} column: {} not exists", catalogId, dbID, tblId, colName);
return PartitionColumnStatistic.UNKNOWN;
}

PartitionColumnStatisticBuilder partitionStatisticBuilder = new PartitionColumnStatisticBuilder();
double count = Double.parseDouble(row.get(5));
partitionStatisticBuilder.setCount(count);
String ndv = row.get(6);
Base64.Decoder decoder = Base64.getDecoder();
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(decoder.decode(ndv)));
Hll hll = new Hll();
if (!hll.deserialize(dis)) {
LOG.warn("Failed to deserialize ndv. [{}]", row);
return PartitionColumnStatistic.UNKNOWN;
}
partitionStatisticBuilder.setNdv(Hll128.fromHll(hll));
String nullCount = row.getWithDefault(7, "0");
partitionStatisticBuilder.setNumNulls(Double.parseDouble(nullCount));
partitionStatisticBuilder.setDataSize(Double
.parseDouble(row.getWithDefault(10, "0")));
partitionStatisticBuilder.setAvgSizeByte(partitionStatisticBuilder.getCount() == 0
? 0 : partitionStatisticBuilder.getDataSize()
/ partitionStatisticBuilder.getCount());
String min = row.get(8);
String max = row.get(9);
if (!"NULL".equalsIgnoreCase(min)) {
try {
partitionStatisticBuilder.setMinValue(StatisticsUtil.convertToDouble(col.getType(), min));
partitionStatisticBuilder.setMinExpr(StatisticsUtil.readableValue(col.getType(), min));
} catch (AnalysisException e) {
LOG.warn("Failed to deserialize column {} min value {}.", col, min, e);
partitionStatisticBuilder.setMinValue(Double.NEGATIVE_INFINITY);
}
} else {
partitionStatisticBuilder.setMinValue(Double.NEGATIVE_INFINITY);
}
if (!"NULL".equalsIgnoreCase(max)) {
try {
partitionStatisticBuilder.setMaxValue(StatisticsUtil.convertToDouble(col.getType(), max));
partitionStatisticBuilder.setMaxExpr(StatisticsUtil.readableValue(col.getType(), max));
} catch (AnalysisException e) {
LOG.warn("Failed to deserialize column {} max value {}.", col, max, e);
partitionStatisticBuilder.setMaxValue(Double.POSITIVE_INFINITY);
}
} else {
partitionStatisticBuilder.setMaxValue(Double.POSITIVE_INFINITY);
}
partitionStatisticBuilder.setUpdatedTime(row.get(11));
return partitionStatisticBuilder.build();
} catch (Exception e) {
LOG.warn("Failed to deserialize column statistics. Row [{}]", row, e);
return PartitionColumnStatistic.UNKNOWN;
}
}
}
Loading

0 comments on commit cc4a447

Please sign in to comment.