-
Notifications
You must be signed in to change notification settings - Fork 4.8k
HIVE-29368: more conservative NDV combining by PessimisticStatCombiner #6244
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
633951c
199c441
f0022f7
bd86e3c
75dbdf8
0ddef8c
8b361de
1285297
40fc7ff
cf4fa0b
5bd21e6
c18f8cd
bb7c3fd
bdc395f
459e85f
b59cc9d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -832,6 +832,7 @@ public static ColStatistics getColStatistics(ColumnStatisticsObj cso, String col | |
| cs.setNumNulls(csd.getBinaryStats().getNumNulls()); | ||
| } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME)) { | ||
| cs.setAvgColLen(JavaDataModel.get().lengthOfTimestamp()); | ||
| cs.setCountDistint(csd.getTimestampStats().getNumDVs()); | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am unsure if this was deliberately not added or an unintended omission. It does seem to improve stats' calculations of multiple .q test files, especially after more conservative NDV handling by PessimisticStatCombiner |
||
| cs.setNumNulls(csd.getTimestampStats().getNumNulls()); | ||
| Long lowVal = (csd.getTimestampStats().getLowValue() != null) ? csd.getTimestampStats().getLowValue() | ||
| .getSecondsSinceEpoch() : null; | ||
|
|
@@ -862,6 +863,7 @@ public static ColStatistics getColStatistics(ColumnStatisticsObj cso, String col | |
| cs.setHistogram(csd.getDecimalStats().getHistogram()); | ||
| } else if (colTypeLowerCase.equals(serdeConstants.DATE_TYPE_NAME)) { | ||
| cs.setAvgColLen(JavaDataModel.get().lengthOfDate()); | ||
| cs.setCountDistint(csd.getDateStats().getNumDVs()); | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am unsure if this was deliberately not added or an unintended omission. It does seem to improve stats' calculations of multiple .q test files, especially after more conservative NDV handling by PessimisticStatCombiner |
||
| cs.setNumNulls(csd.getDateStats().getNumNulls()); | ||
| Long lowVal = (csd.getDateStats().getLowValue() != null) ? csd.getDateStats().getLowValue() | ||
| .getDaysSinceEpoch() : null; | ||
|
|
@@ -2086,11 +2088,7 @@ private static List<Long> extractNDVGroupingColumns(List<ColStatistics> colStats | |
| // compute product of distinct values of grouping columns | ||
| for (ColStatistics cs : colStats) { | ||
| if (cs != null) { | ||
| long ndv = cs.getCountDistint(); | ||
| if (cs.getNumNulls() > 0) { | ||
| ndv = StatsUtils.safeAdd(ndv, 1); | ||
| } | ||
| ndvValues.add(ndv); | ||
| ndvValues.add(getGroupingColumnNdv(cs, parentStats)); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please create a separate pull request next time you update something with global impact. This PR affects approximately 200 test cases and would make it harder for a reviewer to validate them if it included two or more types of changes.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. After taking a glance at all test files, I started feeling I would like to separate unrelated changes, like below.
This is because I can review each of them in 30 minutes if they are separated, so I will spend only 90 minutes in total. If all are included, it is not very obvious why each test case has changed. I need more focus, and we can't make a checkpoint because we can't merge it unless all changes are reasonable and all test cases are green (I know some integration tests are still failing and Sonar Cloud is reporting some remaining issues). This proposal is negotiable because it requires your efforts. I should have proposed it at the beginning. |
||
| } else { | ||
| if (parentStats.getColumnStatsState().equals(Statistics.State.COMPLETE)) { | ||
| // the column must be an aggregate column inserted by GBY. We | ||
|
|
@@ -2109,4 +2107,20 @@ private static List<Long> extractNDVGroupingColumns(List<ColStatistics> colStats | |
|
|
||
| return ndvValues; | ||
| } | ||
|
|
||
| private static long getGroupingColumnNdv(ColStatistics cs, Statistics parentStats) { | ||
| long ndv = cs.getCountDistint(); | ||
|
|
||
| if (ndv == 0L) { | ||
| // Typically, ndv == 0 means "NDV unknown", and no safe GROUPBY adjustments are possible | ||
| // However, there is a special exception for "constant NULL" columns. They are intentionally generated | ||
| // with NDV values of 0 and numNulls == numRows, while their actual NDV is 1 | ||
| if (cs.getNumNulls() >= parentStats.getNumRows()) { | ||
| ndv = 1L; | ||
| } | ||
| } else if (cs.getNumNulls() > 0L) { | ||
| ndv = StatsUtils.safeAdd(ndv, 1L); | ||
| } | ||
| return ndv; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,52 @@ | ||
| /* | ||
| * 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.hadoop.hive.ql.stats.estimator; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Optional; | ||
|
|
||
| import org.apache.hadoop.hive.ql.plan.ColStatistics; | ||
|
|
||
| /** | ||
| * Base class for StatEstimators that handle branching expressions (CASE/WHEN, IF, COALESCE). | ||
| * Combines branch statistics using PessimisticStatCombiner and accounts for distinct constants. | ||
| */ | ||
| public abstract class BranchingStatEstimator implements StatEstimator { | ||
| protected final int numberOfDistinctConstants; | ||
|
|
||
| protected BranchingStatEstimator(int numberOfDistinctConstants) { | ||
| this.numberOfDistinctConstants = numberOfDistinctConstants; | ||
| } | ||
|
|
||
| @Override | ||
| public Optional<ColStatistics> estimate(List<ColStatistics> argStats) { | ||
| PessimisticStatCombiner combiner = new PessimisticStatCombiner(); | ||
| addBranchStats(combiner, argStats); | ||
| Optional<ColStatistics> result = combiner.getResult(); | ||
| if (result.isPresent()) { | ||
| ColStatistics stat = result.get(); | ||
| if (numberOfDistinctConstants > stat.getCountDistint() && stat.getCountDistint() > 0) { | ||
| stat.setCountDistint(numberOfDistinctConstants); | ||
| } | ||
| } | ||
| return result; | ||
| } | ||
|
|
||
| protected abstract void addBranchStats(PessimisticStatCombiner combiner, List<ColStatistics> argStats); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -37,31 +37,30 @@ public void add(ColStatistics stat) { | |
| result.setRange(null); | ||
| result.setIsEstimated(true); | ||
| return; | ||
| } else { | ||
| if (stat.getAvgColLen() > result.getAvgColLen()) { | ||
| result.setAvgColLen(stat.getAvgColLen()); | ||
| } | ||
| if (stat.getCountDistint() > result.getCountDistint()) { | ||
| result.setCountDistint(stat.getCountDistint()); | ||
| } | ||
| if (stat.getNumNulls() > result.getNumNulls()) { | ||
| result.setNumNulls(stat.getNumNulls()); | ||
| } | ||
| if (stat.getNumTrues() > result.getNumTrues()) { | ||
| result.setNumTrues(stat.getNumTrues()); | ||
| } | ||
| if (stat.getNumFalses() > result.getNumFalses()) { | ||
| result.setNumFalses(stat.getNumFalses()); | ||
| } | ||
| if (stat.isFilteredColumn()) { | ||
| result.setFilterColumn(); | ||
| } | ||
|
|
||
| } | ||
|
|
||
| if (stat.getAvgColLen() > result.getAvgColLen()) { | ||
| result.setAvgColLen(stat.getAvgColLen()); | ||
| } | ||
| if (stat.getCountDistint() == 0 || result.getCountDistint() == 0) { | ||
| result.setCountDistint(0L); | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This could appear counter-intuitive at first, however, when combining statistics of different logical branches of the same column, and having no reliable information about their interdependencies (i.e. in a "truly pessimistic" scenario), every other option appears to introduce undesired under-estimations, which often lead to catastrophic query failures. For example, a simple column generated by an CASE..WHEN clause with three constants produces an NDV of 1 by the original code, while, in most cases, the "true" NDV is 3. If such a column participates in a GROUP BY condition later on, its estimated number of records naturally becomes "1". Even this seemingly small under-estimation could lead to bad decision of converting to a mapjoin or not, especially over large data sets. Alternatively, trying to "total up" NDV values of the same columns could cause over-estimation of the true NDV of such a column, which, it its turn, could lead to a severe underestimation of records matching an "IN" filter, ultimately producing equally bad results as the previous case
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Edit: per the PR feedback, this has been refined to only set NDV to "Unknown" if either part of the combined values is also "Unknown", resulting in much better estimates |
||
| } else if (stat.getCountDistint() > result.getCountDistint()) { | ||
| result.setCountDistint(stat.getCountDistint()); | ||
| } | ||
| if (stat.getNumNulls() > result.getNumNulls()) { | ||
| result.setNumNulls(stat.getNumNulls()); | ||
| } | ||
| if (stat.getNumTrues() > result.getNumTrues()) { | ||
| result.setNumTrues(stat.getNumTrues()); | ||
| } | ||
| if (stat.getNumFalses() > result.getNumFalses()) { | ||
| result.setNumFalses(stat.getNumFalses()); | ||
| } | ||
| if (stat.isFilteredColumn()) { | ||
| result.setFilterColumn(); | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we need to change this method? I'm expecting stat = result here
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is not something I actually changed; the diff shows up because I've removed some pre-existing empty lines per the Quality Gate feedback |
||
| } | ||
|
|
||
| public Optional<ColStatistics> getResult() { | ||
| return Optional.of(result); | ||
|
|
||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Note: This is probably ok but I want to check it again
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@okumin I am unsure I fully understand this comment, could you please provide more info?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@konstantinb Sorry for confusing you. This is a comment for myself. I took a glance at this code, and it seems to be OK, but I have not dug into the entire semantics of JoinStatsRule. I can't merge an OSS pull request with a very optimistic imagination. So, I want to take a deep look again later. I'll write this sort of info in my private note next time. Sorry