diff --git ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java index a718264..55650d1 100644 --- ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java @@ -30,12 +30,10 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -100,6 +98,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; import org.apache.hadoop.io.BytesWritable; import org.apache.hive.common.util.AnnotationUtils; @@ -110,6 +109,7 @@ import com.google.common.base.Joiner; import com.google.common.collect.Lists; import com.google.common.math.LongMath; +import com.google.common.util.concurrent.ThreadFactoryBuilder; public class StatsUtils { @@ -1307,11 +1307,33 @@ public static ColStatistics getColStatisticsFromExpression(HiveConf conf, Statis countDistincts = 1; } } else if (end instanceof ExprNodeGenericFuncDesc) { - - // udf projection ExprNodeGenericFuncDesc engfd = (ExprNodeGenericFuncDesc) end; colName = engfd.getName(); - colType = engfd.getTypeString(); + colType = engfd.getTypeString().toLowerCase(); + + // If it is a widening cast, we do not change NDV, min, max + if (isWideningCast(engfd)) { + if (engfd.getCols().size() != 1) { + // Bail out to default + LOG.warn(engfd + " is as CAST, but contains more than one column"); + } else { + ColStatistics stats = parentStats.getColumnStatisticsFromColName(engfd.getCols().get(0)); + if (stats != null) { + ColStatistics newStats; + try { + newStats = stats.clone(); + } catch (CloneNotSupportedException e) { + LOG.warn("error cloning stats, this should not happen"); + return null; + } + newStats.setColumnName(colName); + newStats.setColumnType(colType); + newStats.setAvgColLen(getAvgColLenOf(conf, oi, colType)); + return newStats; + } + } + } + countDistincts = getNDVFor(engfd, numRows, parentStats); } else if (end instanceof ExprNodeColumnListDesc) { @@ -1341,6 +1363,15 @@ public static ColStatistics getColStatisticsFromExpression(HiveConf conf, Statis return colStats; } + private static boolean isWideningCast(ExprNodeGenericFuncDesc engfd) { + GenericUDF udf = engfd.getGenericUDF(); + if (!FunctionRegistry.isOpCast(udf)) { + // It is not a cast + return false; + } + return TypeInfoUtils.implicitConvertible(engfd.getChildren().get(0).getTypeInfo(), + engfd.getTypeInfo()); + } public static Long addWithExpDecay (List distinctVals) { // Exponential back-off for NDVs.