You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by rh...@apache.org on 2014/08/30 03:11:21 UTC
svn commit: r1621408 - in
/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql:
optimizer/optiq/RelOptHiveTable.java
optimizer/optiq/translator/RelNodeConverter.java
optimizer/optiq/translator/TypeConverter.java parse/SemanticAnalyzer.java
Author: rhbutani
Date: Sat Aug 30 01:11:20 2014
New Revision: 1621408
URL: http://svn.apache.org/r1621408
Log:
HIVE-7918 CBO: String Type Conversion, Stats error Propagation, CBO Patch check (John Pullokkaran via Harish Butani)
Modified:
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/RelOptHiveTable.java
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/RelNodeConverter.java
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/TypeConverter.java
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
Modified: hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/RelOptHiveTable.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/RelOptHiveTable.java?rev=1621408&r1=1621407&r2=1621408&view=diff
==============================================================================
--- hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/RelOptHiveTable.java (original)
+++ hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/RelOptHiveTable.java Sat Aug 30 01:11:20 2014
@@ -7,6 +7,8 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -48,6 +50,7 @@ public class RelOptHiveTable extends Rel
private Integer m_numPartitions;
PrunedPartitionList partitionList;
Map<String, PrunedPartitionList> partitionCache;
+ AtomicInteger noColsMissingStats;
protected static final Log LOG = LogFactory
.getLog(RelOptHiveTable.class
@@ -55,7 +58,7 @@ public class RelOptHiveTable extends Rel
public RelOptHiveTable(RelOptSchema optiqSchema, String name, RelDataType rowType,
Table hiveTblMetadata, List<ColumnInfo> hiveNonPartitionCols,
- List<ColumnInfo> hivePartitionCols, HiveConf hconf, Map<String, PrunedPartitionList> partitionCache) {
+ List<ColumnInfo> hivePartitionCols, HiveConf hconf, Map<String, PrunedPartitionList> partitionCache, AtomicInteger noColsMissingStats) {
super(optiqSchema, name, rowType);
m_hiveTblMetadata = hiveTblMetadata;
m_hiveNonPartitionCols = ImmutableList.copyOf(hiveNonPartitionCols);
@@ -64,6 +67,7 @@ public class RelOptHiveTable extends Rel
m_noOfProjs = hiveNonPartitionCols.size() + hivePartitionCols.size();
m_hiveConf = hconf;
this.partitionCache = partitionCache;
+ this.noColsMissingStats = noColsMissingStats;
}
private static ImmutableMap<Integer, ColumnInfo> getColInfoMap(List<ColumnInfo> hiveCols,
@@ -264,6 +268,7 @@ public class RelOptHiveTable extends Rel
String logMsg = "No Stats for " + m_hiveTblMetadata.getCompleteName() + ", Columns: "
+ getColNamesForLogging(colNamesFailedStats);
LOG.error(logMsg);
+ noColsMissingStats.getAndAdd(colNamesFailedStats.size());
throw new RuntimeException(logMsg);
}
}
Modified: hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/RelNodeConverter.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/RelNodeConverter.java?rev=1621408&r1=1621407&r2=1621408&view=diff
==============================================================================
--- hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/RelNodeConverter.java (original)
+++ hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/RelNodeConverter.java Sat Aug 30 01:11:20 2014
@@ -644,7 +644,7 @@ public class RelNodeConverter {
}
RelDataType rowType = TypeConverter.getType(ctx.cluster, rr, neededCols);
RelOptHiveTable optTable = new RelOptHiveTable(ctx.schema, tableScanOp.getConf().getAlias(),
- rowType, ctx.sA.getTable(tableScanOp), null, null, null, null);
+ rowType, ctx.sA.getTable(tableScanOp), null, null, null, null, null);
TableAccessRelBase tableRel = new HiveTableScanRel(ctx.cluster,
ctx.cluster.traitSetOf(HiveRel.CONVENTION), optTable, rowType);
ctx.buildColumnMap(tableScanOp, tableRel);
Modified: hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/TypeConverter.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/TypeConverter.java?rev=1621408&r1=1621407&r2=1621408&view=diff
==============================================================================
--- hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/TypeConverter.java (original)
+++ hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/TypeConverter.java Sat Aug 30 01:11:20 2014
@@ -9,6 +9,7 @@ import org.apache.hadoop.hive.ql.exec.Ro
import org.apache.hadoop.hive.ql.optimizer.optiq.translator.SqlFunctionConverter.HiveToken;
import org.apache.hadoop.hive.ql.parse.HiveParser;
import org.apache.hadoop.hive.ql.parse.RowResolver;
+import org.apache.hadoop.hive.serde.serdeConstants;
import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
@@ -135,7 +136,8 @@ public class TypeConverter {
break;
case STRING:
//TODO: shall we pass -1 for len to distinguish between STRING & VARCHAR on way out
- convertedType = dtFactory.createSqlType(SqlTypeName.VARCHAR, 1);
+ convertedType = dtFactory.createSqlType(SqlTypeName.VARCHAR,
+ RelDataType.PRECISION_NOT_SPECIFIED);
break;
case DATE:
convertedType = dtFactory.createSqlType(SqlTypeName.DATE);
@@ -269,7 +271,10 @@ public class TypeConverter {
case DECIMAL:
return TypeInfoFactory.getDecimalTypeInfo(rType.getPrecision(), rType.getScale());
case VARCHAR:
- return TypeInfoFactory.getVarcharTypeInfo(rType.getPrecision());
+ if (rType.getPrecision() == RelDataType.PRECISION_NOT_SPECIFIED)
+ return TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.STRING_TYPE_NAME);
+ else
+ return TypeInfoFactory.getVarcharTypeInfo(rType.getPrecision());
case CHAR:
return TypeInfoFactory.getCharTypeInfo(rType.getPrecision());
case OTHER:
Modified: hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1621408&r1=1621407&r2=1621408&view=diff
==============================================================================
--- hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Sat Aug 30 01:11:20 2014
@@ -36,6 +36,8 @@ import java.util.Map.Entry;
import java.util.Set;
import java.util.TreeSet;
import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
@@ -1042,10 +1044,15 @@ public class SemanticAnalyzer extends Ba
qb.countSel();
qbp.setSelExprForClause(ctx_1.dest, ast);
+ int posn = 0;
if (((ASTNode) ast.getChild(0)).getToken().getType() == HiveParser.TOK_HINTLIST) {
qbp.setHints((ASTNode) ast.getChild(0));
+ posn++;
}
+ if ((ast.getChild(posn).getChild(0).getType() == HiveParser.TOK_TRANSFORM))
+ queryProperties.setUsesScript(true);
+
LinkedHashMap<String, ASTNode> aggregations = doPhase1GetAggregationsFromSelect(ast,
qb, ctx_1.dest);
doPhase1GetColumnAliasesFromSelect(ast, qbp);
@@ -9582,11 +9589,12 @@ public class SemanticAnalyzer extends Ba
Operator sinkOp = null;
if (runCBO) {
+ OptiqBasedPlanner optiqPlanner = new OptiqBasedPlanner();
boolean reAnalyzeAST = false;
try {
// 1. Gen Optimized AST
- ASTNode newAST = new OptiqBasedPlanner().getOptimizedAST(prunedPartitions);
+ ASTNode newAST = optiqPlanner.getOptimizedAST(prunedPartitions);
// 2. Regen OP plan from optimized AST
init(false);
@@ -9614,10 +9622,10 @@ public class SemanticAnalyzer extends Ba
* .getRowResolver(), true);
*/
} catch (Exception e) {
- //TODO: Distinguish between exceptions that can be retried vs user errors
LOG.error("CBO failed, skipping CBO. ", e);
- if (!conf.getBoolVar(ConfVars.HIVE_IN_TEST))
+ if (!conf.getBoolVar(ConfVars.HIVE_IN_TEST) || (optiqPlanner.noColsMissingStats.get() > 0)) {
reAnalyzeAST = true;
+ }
} finally {
runCBO = false;
disableJoinMerge = false;
@@ -11811,14 +11819,13 @@ public class SemanticAnalyzer extends Ba
private boolean canHandleQuery() {
boolean runOptiqPlanner = false;
- if ( conf.getBoolVar(ConfVars.HIVE_IN_TEST) || (queryProperties.getJoinCount() > 1)
+ if (((queryProperties.getJoinCount() > 1) || conf.getBoolVar(ConfVars.HIVE_IN_TEST))
&& !queryProperties.hasClusterBy()
&& !queryProperties.hasDistributeBy()
&& !queryProperties.hasSortBy()
&& !queryProperties.hasPTF()
&& !queryProperties.usesScript()
- && !queryProperties.hasMultiDestQuery()
- && !queryProperties.hasFilterWithSubQuery()) {
+ && !queryProperties.hasMultiDestQuery()) {
runOptiqPlanner = true;
} else {
LOG.info("Can not invoke CBO; query contains operators not supported for CBO.");
@@ -11832,6 +11839,7 @@ public class SemanticAnalyzer extends Ba
RelOptSchema m_relOptSchema;
SemanticException m_semanticException;
Map<String, PrunedPartitionList> partitionCache;
+ AtomicInteger noColsMissingStats = new AtomicInteger(0);
// TODO: Do we need to keep track of RR, ColNameToPosMap for every op or
// just last one.
@@ -12350,7 +12358,7 @@ public class SemanticAnalyzer extends Ba
// 4. Build RelOptAbstractTable
RelOptHiveTable optTable = new RelOptHiveTable(m_relOptSchema, tableAlias, rowType, tab,
- nonPartitionColumns, partitionColumns, conf, partitionCache);
+ nonPartitionColumns, partitionColumns, conf, partitionCache, noColsMissingStats);
// 5. Build Hive Table Scan Rel
tableRel = new HiveTableScanRel(m_cluster, m_cluster.traitSetOf(HiveRel.CONVENTION),