You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2011/04/19 20:18:21 UTC
svn commit: r1095153 [1/2] - in /hive/trunk:
cli/src/java/org/apache/hadoop/hive/cli/
common/src/java/org/apache/hadoop/hive/conf/ conf/
hwi/src/java/org/apache/hadoop/hive/hwi/
ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/...
Author: namit
Date: Tue Apr 19 18:18:20 2011
New Revision: 1095153
URL: http://svn.apache.org/viewvc?rev=1095153&view=rev
Log:
HIVE-2068 Speed up a small limit query
(Siying Dong via namit)
Added:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java
hive/trunk/ql/src/test/queries/clientpositive/global_limit.q
hive/trunk/ql/src/test/results/clientpositive/global_limit.q.out
Modified:
hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
hive/trunk/conf/hive-default.xml
hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Context.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/LimitDesc.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java
hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
hive/trunk/service/src/java/org/apache/hadoop/hive/service/HiveServer.java
Modified: hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Tue Apr 19 18:18:20 2011
@@ -32,12 +32,12 @@ import java.util.Map;
import java.util.Set;
import jline.ArgumentCompletor;
-import jline.ArgumentCompletor.AbstractArgumentDelimiter;
-import jline.ArgumentCompletor.ArgumentDelimiter;
import jline.Completor;
import jline.ConsoleReader;
import jline.History;
import jline.SimpleCompletor;
+import jline.ArgumentCompletor.AbstractArgumentDelimiter;
+import jline.ArgumentCompletor.ArgumentDelimiter;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
@@ -46,6 +46,7 @@ import org.apache.hadoop.conf.Configurat
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
import org.apache.hadoop.hive.ql.Driver;
import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -198,71 +199,85 @@ public class CliDriver {
}
} else { // local mode
CommandProcessor proc = CommandProcessorFactory.get(tokens[0], (HiveConf)conf);
- if (proc != null) {
- if (proc instanceof Driver) {
- Driver qp = (Driver) proc;
- PrintStream out = ss.out;
- long start = System.currentTimeMillis();
- if (ss.getIsVerbose()) {
- out.println(cmd);
- }
-
- ret = qp.run(cmd).getResponseCode();
- if (ret != 0) {
- qp.close();
- return ret;
- }
+ int tryCount = 0;
+ boolean needRetry;
- ArrayList<String> res = new ArrayList<String>();
+ do {
+ try {
+ needRetry = false;
+ if (proc != null) {
+ if (proc instanceof Driver) {
+ Driver qp = (Driver) proc;
+ PrintStream out = ss.out;
+ long start = System.currentTimeMillis();
+ if (ss.getIsVerbose()) {
+ out.println(cmd);
+ }
- if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) {
- // Print the column names
- boolean first_col = true;
- Schema sc = qp.getSchema();
- for (FieldSchema fs : sc.getFieldSchemas()) {
- if (!first_col) {
- out.print('\t');
+ qp.setTryCount(tryCount);
+ ret = qp.run(cmd).getResponseCode();
+ if (ret != 0) {
+ qp.close();
+ return ret;
}
- out.print(fs.getName());
- first_col = false;
- }
- out.println();
- }
- try {
- while (qp.getResults(res)) {
- for (String r : res) {
- out.println(r);
+ ArrayList<String> res = new ArrayList<String>();
+
+ if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) {
+ // Print the column names
+ boolean first_col = true;
+ Schema sc = qp.getSchema();
+ for (FieldSchema fs : sc.getFieldSchemas()) {
+ if (!first_col) {
+ out.print('\t');
+ }
+ out.print(fs.getName());
+ first_col = false;
+ }
+ out.println();
}
- res.clear();
- if (out.checkError()) {
- break;
+
+ try {
+ while (qp.getResults(res)) {
+ for (String r : res) {
+ out.println(r);
+ }
+ res.clear();
+ if (out.checkError()) {
+ break;
+ }
+ }
+ } catch (IOException e) {
+ console.printError("Failed with exception " + e.getClass().getName() + ":"
+ + e.getMessage(), "\n"
+ + org.apache.hadoop.util.StringUtils.stringifyException(e));
+ ret = 1;
}
- }
- } catch (IOException e) {
- console.printError("Failed with exception " + e.getClass().getName() + ":"
- + e.getMessage(), "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
- ret = 1;
- }
- int cret = qp.close();
- if (ret == 0) {
- ret = cret;
- }
+ int cret = qp.close();
+ if (ret == 0) {
+ ret = cret;
+ }
- long end = System.currentTimeMillis();
- if (end > start) {
- double timeTaken = (end - start) / 1000.0;
- console.printInfo("Time taken: " + timeTaken + " seconds", null);
- }
+ long end = System.currentTimeMillis();
+ if (end > start) {
+ double timeTaken = (end - start) / 1000.0;
+ console.printInfo("Time taken: " + timeTaken + " seconds", null);
+ }
- } else {
- if (ss.getIsVerbose()) {
- ss.out.println(tokens[0] + " " + cmd_1);
+ } else {
+ if (ss.getIsVerbose()) {
+ ss.out.println(tokens[0] + " " + cmd_1);
+ }
+ ret = proc.run(cmd_1).getResponseCode();
+ }
}
- ret = proc.run(cmd_1).getResponseCode();
+ } catch (CommandNeedRetryException e) {
+ console.printInfo("Retry query with a different approach...");
+ tryCount++;
+ needRetry = true;
}
- }
+ } while (needRetry);
}
return ret;
Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Apr 19 18:18:20 2011
@@ -328,6 +328,10 @@ public class HiveConf extends Configurat
HIVESENDHEARTBEAT("hive.heartbeat.interval", 1000),
HIVEMAXMAPJOINSIZE("hive.mapjoin.maxsize", 100000),
+ HIVELIMITMAXROWSIZE("hive.limit.row.max.size", 100000L),
+ HIVELIMITOPTLIMITFILE("hive.limit.optimize.limit.file", 10),
+ HIVELIMITOPTENABLE("hive.limit.optimize.enable", false),
+ HIVELIMITOPTMAXFETCH("hive.limit.optimize.fetch.max", 50000),
HIVEHASHTABLETHRESHOLD("hive.hashtable.initialCapacity", 100000),
HIVEHASHTABLELOADFACTOR("hive.hashtable.loadfactor", (float) 0.75),
HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE("hive.mapjoin.followby.gby.localtask.max.memory.usage", (float) 0.55),
Modified: hive/trunk/conf/hive-default.xml
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml (original)
+++ hive/trunk/conf/hive-default.xml Tue Apr 19 18:18:20 2011
@@ -998,5 +998,31 @@
<description>A comma separated list of acceptable URI schemes for import and export.</description>
</property>
+<property>
+ <name>hive.limit.row.max.size</name>
+ <value>100000</value>
+ <description>When trying a smaller subset of data for simple LIMIT, how much size we need to guarantee
+ each row to have at least.</description>
+</property>
+
+<property>
+ <name>hive.limit.optimize.limit.file</name>
+ <value>10</value>
+ <description>When trying a smaller subset of data for simple LIMIT, maximum number of files we can
+ sample.</description>
+</property>
+
+<property>
+ <name>hive.limit.optimize.enable</name>
+ <value>false</value>
+ <description>Whether to enable to optimization to trying a smaller subset of data for simple LIMIT first.</description>
+</property>
+
+<property>
+ <name>hive.limit.optimize.fetch.max</name>
+ <value>50000</value>
+ <description>Maximum number of rows allowed for a smaller subset of data for simple LIMIT, if it is a fetch query.
+ Insert queries are not restricted by this limit.</description>
+</property>
</configuration>
Modified: hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java
URL: http://svn.apache.org/viewvc/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java (original)
+++ hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java Tue Apr 19 18:18:20 2011
@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFac
import org.apache.hadoop.hive.cli.CliSessionState;
import org.apache.hadoop.hive.cli.OptionsProcessor;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
import org.apache.hadoop.hive.ql.Driver;
import org.apache.hadoop.hive.ql.history.HiveHistoryViewer;
import org.apache.hadoop.hive.ql.processors.CommandProcessor;
@@ -39,7 +40,7 @@ import org.apache.hadoop.hive.ql.session
/**
* HWISessionItem can be viewed as a wrapper for a Hive shell. With it the user
* has a session on the web server rather then in a console window.
- *
+ *
*/
public class HWISessionItem implements Runnable, Comparable<HWISessionItem> {
@@ -215,7 +216,7 @@ public class HWISessionItem implements R
/**
* Helper function to get configuration variables.
- *
+ *
* @param wanted
* a ConfVar
* @return Value of the configuration variable.
@@ -330,7 +331,9 @@ public class HWISessionItem implements R
if (proc != null) {
if (proc instanceof Driver) {
Driver qp = (Driver) proc;
- queryRet.add(new Integer(qp.run(cmd).getResponseCode()));
+ qp.setTryCount(Integer.MAX_VALUE);
+ try {
+ queryRet.add(new Integer(qp.run(cmd).getResponseCode()));
ArrayList<String> res = new ArrayList<String>();
try {
while (qp.getResults(res)) {
@@ -351,13 +354,24 @@ public class HWISessionItem implements R
}
res.clear();
}
+
} catch (IOException ex) {
l4j.error(getSessionName() + " getting results " + getResultFile()
+ " caused exception.", ex);
}
- qp.close();
+ } catch (CommandNeedRetryException e) {
+ // this should never happen since we Driver.setTryCount(Integer.MAX_VALUE)
+ l4j.error(getSessionName() + " Exception when executing", e);
+ } finally {
+ qp.close();
+ }
} else {
- queryRet.add(new Integer(proc.run(cmd_1).getResponseCode()));
+ try {
+ queryRet.add(new Integer(proc.run(cmd_1).getResponseCode()));
+ } catch (CommandNeedRetryException e) {
+ // this should never happen if there is no bug
+ l4j.error(getSessionName() + " Exception when executing", e);
+ }
}
} else {
// processor was null
@@ -412,7 +426,7 @@ public class HWISessionItem implements R
}
/**
- *
+ *
* @return the HiveHistoryViewer for the session
* @throws HWIException
*/
@@ -430,7 +444,7 @@ public class HWISessionItem implements R
/**
* Uses the sessionName property to compare to sessions.
- *
+ *
* @return true if sessionNames are equal false otherwise
*/
@Override
@@ -459,7 +473,7 @@ public class HWISessionItem implements R
/**
* The session name is an identifier to recognize the session.
- *
+ *
* @return the session's name
*/
public String getSessionName() {
@@ -470,7 +484,7 @@ public class HWISessionItem implements R
* Used to represent to the user and other components what state the
* HWISessionItem is in. Certain commands can only be run when the application
* is in certain states.
- *
+ *
* @return the current status of the session
*/
public WebSessionItemStatus getStatus() {
@@ -479,7 +493,7 @@ public class HWISessionItem implements R
/**
* Currently unused.
- *
+ *
* @return a String with the full path to the error file.
*/
public String getErrorFile() {
@@ -488,7 +502,7 @@ public class HWISessionItem implements R
/**
* Currently unused.
- *
+ *
* @param errorFile
* the full path to the file for results.
*/
@@ -518,7 +532,7 @@ public class HWISessionItem implements R
/**
* Adds a new query to the execution list.
- *
+ *
* @param query
* query to be added to the list
*/
@@ -529,7 +543,7 @@ public class HWISessionItem implements R
/**
* Removes a query from the execution list.
- *
+ *
* @param item
* the 0 based index of the item to be removed
*/
@@ -550,7 +564,7 @@ public class HWISessionItem implements R
/**
* sets the value for resultBucketMaxSize.
- *
+ *
* @param size
* the new size
*/
@@ -565,7 +579,7 @@ public class HWISessionItem implements R
/**
* The HWISessionItem stores the result of each query in an array.
- *
+ *
* @return unmodifiable list of return codes
*/
public List<Integer> getQueryRet() {
Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java?rev=1095153&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java Tue Apr 19 18:18:20 2011
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hive.ql;
+
+public class CommandNeedRetryException extends Exception {
+ public CommandNeedRetryException() {
+ super();
+ }
+
+ public CommandNeedRetryException(String message) {
+ super(message);
+ }
+
+ public CommandNeedRetryException(Throwable cause) {
+ super(cause);
+ }
+
+ public CommandNeedRetryException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Context.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Context.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Context.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Context.java Tue Apr 19 18:18:20 2011
@@ -73,6 +73,8 @@ public class Context {
private final Configuration conf;
protected int pathid = 10000;
protected boolean explain = false;
+ // number of previous attempts
+ protected int tryCount = 0;
private TokenRewriteStream tokenRewriteStream;
String executionId;
@@ -554,4 +556,12 @@ public class Context {
public void setNeedLockMgr(boolean needLockMgr) {
this.needLockMgr = needLockMgr;
}
+
+ public int getTryCount() {
+ return tryCount;
+ }
+
+ public void setTryCount(int tryCount) {
+ this.tryCount = tryCount;
+ }
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Tue Apr 19 18:18:20 2011
@@ -127,6 +127,8 @@ public class Driver implements CommandPr
// A limit on the number of threads that can be launched
private int maxthreads;
private final int sleeptime = 2000;
+ protected int tryCount = Integer.MAX_VALUE;
+
private int checkLockManager() {
boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
@@ -326,6 +328,7 @@ public class Driver implements CommandPr
try {
command = new VariableSubstitution().substitute(conf,command);
ctx = new Context(conf);
+ ctx.setTryCount(getTryCount());
ParseDriver pd = new ParseDriver();
ASTNode tree = pd.parse(command, ctx);
@@ -810,7 +813,7 @@ public class Driver implements CommandPr
Utilities.PerfLogEnd(LOG, "releaseLocks");
}
- public CommandProcessorResponse run(String command) {
+ public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
errorMessage = null;
SQLState = null;
@@ -910,7 +913,7 @@ public class Driver implements CommandPr
return pehooks;
}
- public int execute() {
+ public int execute() throws CommandNeedRetryException {
Utilities.PerfLogBegin(LOG, "Driver.execute");
boolean noName = StringUtils.isEmpty(conf.getVar(HiveConf.ConfVars.HADOOPJOBNAME));
@@ -1001,6 +1004,15 @@ public class Driver implements CommandPr
int exitVal = tskRes.getExitVal();
if (exitVal != 0) {
+ if (tsk.ifRetryCmdWhenFail()) {
+ if (running.size() != 0) {
+ taskCleanup();
+ }
+ // in case we decided to run everything in local mode, restore the
+ // the jobtracker setting to its initial value
+ ctx.restoreOriginalTracker();
+ throw new CommandNeedRetryException();
+ }
Task<? extends Serializable> backupTask = tsk.getAndInitBackupTask();
if (backupTask != null) {
errorMessage = "FAILED: Execution Error, return code " + exitVal + " from "
@@ -1091,6 +1103,8 @@ public class Driver implements CommandPr
String.valueOf(0));
SessionState.get().getHiveHistory().printRowCount(queryId);
}
+ } catch (CommandNeedRetryException e) {
+ throw e;
} catch (Exception e) {
ctx.restoreOriginalTracker();
if (SessionState.get() != null) {
@@ -1214,7 +1228,7 @@ public class Driver implements CommandPr
}
}
- public boolean getResults(ArrayList<String> res) throws IOException {
+ public boolean getResults(ArrayList<String> res) throws IOException, CommandNeedRetryException {
if (plan != null && plan.getFetchTask() != null) {
FetchTask ft = plan.getFetchTask();
ft.setMaxRows(maxRows);
@@ -1267,6 +1281,15 @@ public class Driver implements CommandPr
return true;
}
+ public int getTryCount() {
+ return tryCount;
+ }
+
+ public void setTryCount(int tryCount) {
+ this.tryCount = tryCount;
+ }
+
+
public int close() {
try {
if (plan != null) {
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java Tue Apr 19 18:18:20 2011
@@ -25,6 +25,7 @@ import java.util.Properties;
import org.apache.hadoop.hive.common.JavaUtils;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
import org.apache.hadoop.hive.ql.Context;
import org.apache.hadoop.hive.ql.DriverContext;
import org.apache.hadoop.hive.ql.QueryPlan;
@@ -117,10 +118,27 @@ public class FetchTask extends Task<Fetc
}
@Override
- public boolean fetch(ArrayList<String> res) throws IOException {
+ public boolean fetch(ArrayList<String> res) throws IOException, CommandNeedRetryException {
try {
int numRows = 0;
int rowsRet = maxRows;
+
+ if (work.getLeastNumRows() > 0) {
+ if (totalRows == work.getLeastNumRows()) {
+ return false;
+ }
+ for (int i = 0; i < work.getLeastNumRows(); i++) {
+ InspectableObject io = ftOp.getNextRow();
+ if (io == null) {
+ throw new CommandNeedRetryException();
+ }
+ res.add(((Text) mSerde.serialize(io.o, io.oi)).toString());
+ numRows++;
+ }
+ totalRows = work.getLeastNumRows();
+ return true;
+ }
+
if ((work.getLimit() >= 0) && ((work.getLimit() - totalRows) < rowsRet)) {
rowsRet = work.getLimit() - totalRows;
}
@@ -144,6 +162,8 @@ public class FetchTask extends Task<Fetc
}
totalRows += numRows;
return true;
+ } catch (CommandNeedRetryException e) {
+ throw e;
} catch (IOException e) {
throw e;
} catch (Exception e) {
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java Tue Apr 19 18:18:20 2011
@@ -32,12 +32,14 @@ public class LimitOperator extends Opera
private static final long serialVersionUID = 1L;
protected transient int limit;
+ protected transient int leastRow;
protected transient int currCount;
@Override
protected void initializeOp(Configuration hconf) throws HiveException {
super.initializeOp(hconf);
limit = conf.getLimit();
+ leastRow = conf.getLeastRows();
currCount = 0;
}
@@ -61,4 +63,11 @@ public class LimitOperator extends Opera
return OperatorType.LIMIT;
}
+ @Override
+ public void closeOp(boolean abort) throws HiveException {
+ if (currCount < leastRow) {
+ throw new HiveException("No sufficient row found");
+ }
+ }
+
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java Tue Apr 19 18:18:20 2011
@@ -28,6 +28,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
import org.apache.hadoop.hive.ql.Context;
import org.apache.hadoop.hive.ql.DriverContext;
import org.apache.hadoop.hive.ql.QueryPlan;
@@ -63,6 +64,7 @@ public abstract class Task<T extends Ser
protected List<Task<? extends Serializable>> backupChildrenTasks = new ArrayList<Task<? extends Serializable>>();
protected int taskTag;
private boolean isLocalMode =false;
+ private boolean retryCmdWhenFail = false;
public static final int NO_TAG = 0;
public static final int COMMON_JOIN = 1;
@@ -146,7 +148,7 @@ public abstract class Task<T extends Ser
protected abstract int execute(DriverContext driverContext);
// dummy method - FetchTask overwrites this
- public boolean fetch(ArrayList<String> res) throws IOException {
+ public boolean fetch(ArrayList<String> res) throws IOException, CommandNeedRetryException {
assert false;
return false;
}
@@ -472,4 +474,12 @@ public abstract class Task<T extends Ser
public void setLocalMode(boolean isLocalMode) {
this.isLocalMode = isLocalMode;
}
+
+ public boolean ifRetryCmdWhenFail() {
+ return retryCmdWhenFail;
+ }
+
+ public void setRetryCmdWhenFail(boolean retryCmdWhenFail) {
+ this.retryCmdWhenFail = retryCmdWhenFail;
+ }
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java Tue Apr 19 18:18:20 2011
@@ -587,6 +587,33 @@ public final class GenMapRedUtils {
plan.getAliasToPartnInfo().put(alias_id, aliasPartnDesc);
+ long sizeNeeded = Integer.MAX_VALUE;
+ int fileLimit = -1;
+ if (parseCtx.getGlobalLimitCtx().isEnable()) {
+ long sizePerRow = HiveConf.getLongVar(parseCtx.getConf(), HiveConf.ConfVars.HIVELIMITMAXROWSIZE);
+ sizeNeeded = parseCtx.getGlobalLimitCtx().getGlobalLimit() * sizePerRow;
+ // for the optimization that reduce number of input file, we limit number
+ // of files allowed. If more than specific number of files have to be
+ // selected, we skip this optimization. Since having too many files as
+ // inputs can cause unpredictable latency. It's not necessarily to be
+ // cheaper.
+ fileLimit =
+ HiveConf.getIntVar(parseCtx.getConf(), HiveConf.ConfVars.HIVELIMITOPTLIMITFILE);
+
+ if (sizePerRow <= 0 || fileLimit <= 0) {
+ LOG.info("Skip optimization to reduce input size of 'limit'");
+ parseCtx.getGlobalLimitCtx().disableOpt();
+ } else if (parts.isEmpty()) {
+ LOG.info("Empty input: skip limit optimiztion");
+ } else {
+ LOG.info("Try to reduce input size for 'limit' " +
+ "sizeNeeded: " + sizeNeeded +
+ " file limit : " + fileLimit);
+ }
+ }
+ boolean isFirstPart = true;
+ boolean emptyInput = true;
+ boolean singlePartition = (parts.size() == 1);
for (Partition part : parts) {
if (part.getTable().isPartitioned()) {
inputs.add(new ReadEntity(part));
@@ -596,13 +623,54 @@ public final class GenMapRedUtils {
// Later the properties have to come from the partition as opposed
// to from the table in order to support versioning.
- Path[] paths;
+ Path[] paths = null;
sampleDesc sampleDescr = parseCtx.getOpToSamplePruner().get(topOp);
if (sampleDescr != null) {
paths = SamplePruner.prune(part, sampleDescr);
+ parseCtx.getGlobalLimitCtx().disableOpt();
} else {
- paths = part.getPath();
+ // Now we only try the first partition, if the first partition doesn't
+ // contain enough size, we change to normal mode.
+ if (parseCtx.getGlobalLimitCtx().isEnable()) {
+ if (isFirstPart) {
+ long sizeLeft = sizeNeeded;
+ ArrayList<Path> retPathList = new ArrayList<Path>();
+ SamplePruner.LimitPruneRetStatus status = SamplePruner.limitPrune(part, sizeLeft,
+ fileLimit, retPathList);
+ if (status.equals(SamplePruner.LimitPruneRetStatus.NoFile)) {
+ continue;
+ } else if (status.equals(SamplePruner.LimitPruneRetStatus.NotQualify)) {
+ LOG.info("Use full input -- first " + fileLimit + " files are more than "
+ + sizeNeeded
+ + " bytes");
+
+ parseCtx.getGlobalLimitCtx().disableOpt();
+
+ } else {
+ emptyInput = false;
+ paths = new Path[retPathList.size()];
+ int index = 0;
+ for (Path path : retPathList) {
+ paths[index++] = path;
+ }
+ if (status.equals(SamplePruner.LimitPruneRetStatus.NeedAllFiles) && singlePartition) {
+ // if all files are needed to meet the size limit, we disable
+ // optimization. It usually happens for empty table/partition or
+ // table/partition with only one file. By disabling this
+ // optimization, we can avoid retrying the query if there is
+ // not sufficient rows.
+ parseCtx.getGlobalLimitCtx().disableOpt();
+ }
+ }
+ isFirstPart = false;
+ } else {
+ paths = new Path[0];
+ }
+ }
+ if (!parseCtx.getGlobalLimitCtx().isEnable()) {
+ paths = part.getPath();
+ }
}
// is it a partitioned table ?
@@ -633,6 +701,9 @@ public final class GenMapRedUtils {
}
}
}
+ if (emptyInput) {
+ parseCtx.getGlobalLimitCtx().disableOpt();
+ }
Iterator<Path> iterPath = partDir.iterator();
Iterator<PartitionDesc> iterPartnDesc = partDesc.iterator();
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java Tue Apr 19 18:18:20 2011
@@ -18,7 +18,10 @@
package org.apache.hadoop.hive.ql.optimizer;
+import java.io.IOException;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Map;
@@ -26,6 +29,8 @@ import java.util.Stack;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.exec.FilterOperator;
import org.apache.hadoop.hive.ql.exec.TableScanOperator;
@@ -38,6 +43,7 @@ import org.apache.hadoop.hive.ql.lib.Nod
import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
import org.apache.hadoop.hive.ql.lib.Rule;
import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.metadata.Hive;
import org.apache.hadoop.hive.ql.metadata.Partition;
import org.apache.hadoop.hive.ql.parse.ParseContext;
import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -46,7 +52,7 @@ import org.apache.hadoop.hive.ql.plan.Fi
/**
* The transformation step that does sample pruning.
- *
+ *
*/
public class SamplePruner implements Transform {
@@ -85,7 +91,7 @@ public class SamplePruner implements Tra
/*
* (non-Javadoc)
- *
+ *
* @see
* org.apache.hadoop.hive.ql.optimizer.Transform#transform(org.apache.hadoop
* .hive.ql.parse.ParseContext)
@@ -162,7 +168,7 @@ public class SamplePruner implements Tra
/**
* Prunes to get all the files in the partition that satisfy the TABLESAMPLE
* clause.
- *
+ *
* @param part
* The partition to prune
* @return Path[]
@@ -211,4 +217,120 @@ public class SamplePruner implements Tra
return ret;
}
+ /**
+ * Class used for return value of addPath()
+ *
+ */
+ public static class AddPathReturnStatus {
+ public AddPathReturnStatus(boolean hasFile, boolean allFile, long sizeLeft) {
+ this.hasFile = hasFile;
+ this.allFile = allFile;
+ this.sizeLeft = sizeLeft;
+ }
+ // whether the sub-directory has any file
+ public boolean hasFile;
+ // whether all files are not sufficient to reach sizeLeft
+ public boolean allFile;
+ // remaining size needed after putting files in the return path list
+ public long sizeLeft;
+ }
+
+
+ /**
+ * Try to recursively add files in sub-directories into retPathList until
+ * reaching the sizeLeft.
+ * @param fs
+ * @param pathPattern
+ * @param sizeLeft
+ * @param fileLimit
+ * @param retPathList
+ * @return status of the recursive call
+ * @throws IOException
+ */
+ public static AddPathReturnStatus addPath(FileSystem fs, String pathPattern, long sizeLeft, int fileLimit,
+ Collection<Path> retPathList)
+ throws IOException {
+ LOG.info("Path pattern = " + pathPattern);
+ FileStatus srcs[] = fs.globStatus(new Path(pathPattern));
+ Arrays.sort(srcs);
+
+ boolean hasFile = false, allFile = true;
+
+ for (FileStatus src : srcs) {
+ if (sizeLeft <= 0) {
+ allFile = false;
+ break;
+ }
+ if (src.isDir()) {
+ LOG.info("Got directory: " + src.getPath());
+ AddPathReturnStatus ret = addPath(fs, src.getPath().toString() + "/*", sizeLeft,
+ fileLimit, retPathList);
+ if (ret == null) {
+ // not qualify this optimization
+ return null;
+ }
+ sizeLeft = ret.sizeLeft;
+ hasFile |= ret.hasFile;
+ allFile &= ret.allFile;
+ } else {
+ LOG.info("Got file: " + src.getPath());
+ hasFile = true;
+ retPathList.add(src.getPath());
+ sizeLeft -= src.getLen();
+ if (retPathList.size() >= fileLimit && sizeLeft > 0) {
+ return null;
+ }
+ }
+ }
+ return new AddPathReturnStatus(hasFile, allFile, sizeLeft);
+ }
+
+ public enum LimitPruneRetStatus {
+ // no files in the partition
+ NoFile,
+ // sum size of all files in the partition is smaller than size required
+ NeedAllFiles,
+ // a susbset of files for the partition are sufficient for the optimization
+ NeedSomeFiles,
+ // the partition doesn't qualify the global limit optimization for some reason
+ NotQualify
+ }
+
+
+ /**
+ * Try to generate a list of subset of files in the partition to reach a size
+ * limit with number of files less than fileLimit
+ * @param part
+ * @param sizeLimit
+ * @param fileLimit
+ * @param retPathList list of Paths returned
+ * @return the result of the attempt
+ * @throws SemanticException
+ */
+ public static LimitPruneRetStatus limitPrune(Partition part, long sizeLimit, int fileLimit,
+ Collection<Path> retPathList)
+ throws SemanticException {
+
+ try {
+ FileSystem fs = FileSystem.get(part.getPartitionPath().toUri(), Hive.get()
+ .getConf());
+ String pathPattern = part.getPartitionPath().toString() + "/*";
+ AddPathReturnStatus ret = addPath(fs, pathPattern, sizeLimit, fileLimit, retPathList);
+ if (ret == null) {
+ return LimitPruneRetStatus.NotQualify;
+ } else if (!ret.hasFile){
+ return LimitPruneRetStatus.NoFile;
+ } else if (ret.sizeLeft > 0) {
+ return LimitPruneRetStatus.NotQualify;
+ } else if (ret.allFile) {
+ return LimitPruneRetStatus.NeedAllFiles;
+ } else {
+ return LimitPruneRetStatus.NeedSomeFiles;
+ }
+ } catch (Exception e) {
+ throw new RuntimeException("Cannot get path", e);
+ }
+
+ }
+
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Tue Apr 19 18:18:20 2011
@@ -34,7 +34,6 @@ import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.Map.Entry;
-import java.util.regex.Pattern;
import org.antlr.runtime.tree.CommonTree;
import org.antlr.runtime.tree.Tree;
@@ -73,13 +72,6 @@ import org.apache.hadoop.hive.ql.plan.Al
import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc;
import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
-import org.apache.hadoop.hive.ql.plan.GrantDesc;
-import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
-import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
-import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
-import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
-import org.apache.hadoop.hive.ql.plan.RevokeDesc;
-import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
import org.apache.hadoop.hive.ql.plan.DDLWork;
import org.apache.hadoop.hive.ql.plan.DescDatabaseDesc;
import org.apache.hadoop.hive.ql.plan.DescFunctionDesc;
@@ -88,8 +80,15 @@ import org.apache.hadoop.hive.ql.plan.Dr
import org.apache.hadoop.hive.ql.plan.DropIndexDesc;
import org.apache.hadoop.hive.ql.plan.DropTableDesc;
import org.apache.hadoop.hive.ql.plan.FetchWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
import org.apache.hadoop.hive.ql.plan.LockTableDesc;
import org.apache.hadoop.hive.ql.plan.MsckDesc;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc;
import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc;
import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
@@ -1103,7 +1102,7 @@ public class DDLSemanticAnalyzer extends
rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
alterTblDesc), conf));
}
-
+
private void analyzeAlterTablePartMergeFiles(ASTNode tablePartAST, ASTNode ast,
String tableName, HashMap<String, String> partSpec)
throws SemanticException {
@@ -1115,7 +1114,7 @@ public class DDLSemanticAnalyzer extends
try {
Table tblObj = db.getTable(tableName);
-
+
List<String> bucketCols = null;
Class<? extends InputFormat> inputFormatClass = null;
boolean isArchived = false;
@@ -1167,17 +1166,17 @@ public class DDLSemanticAnalyzer extends
} catch (HiveException e) {
throw new SemanticException(e);
}
-
+
// input and output are the same
inputDir.add(outputDir);
-
+
mergeDesc.setInputDir(inputDir);
mergeDesc.setOutputDir(outputDir);
addInputsOutputsAlterTable(tableName, partSpec);
Task<? extends Serializable> mergeTask = TaskFactory.get(new DDLWork(
getInputs(), getOutputs(), mergeDesc), conf);
-
+
tableSpec tablepart = new tableSpec(this.db, conf, tablePartAST);
StatsWork statDesc = new StatsWork(tablepart);
statDesc.setNoStatsAggregator(true);
@@ -1793,7 +1792,7 @@ public class DDLSemanticAnalyzer extends
rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
addPartitionDesc), conf));
}
-
+
if (isView) {
// Compile internal query to capture underlying table partition
// dependencies
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java Tue Apr 19 18:18:20 2011
@@ -91,6 +91,8 @@ public class ParseContext {
// a map-reduce job
private boolean hasNonPartCols;
+ private SemanticAnalyzer.GlobalLimitCtx globalLimitCtx;
+
public ParseContext() {
}
@@ -143,7 +145,8 @@ public class ParseContext {
UnionProcContext uCtx, List<AbstractMapJoinOperator<? extends MapJoinDesc>> listMapJoinOpsNoReducer,
Map<GroupByOperator, Set<String>> groupOpToInputTables,
Map<String, PrunedPartitionList> prunedPartitions,
- HashMap<TableScanOperator, sampleDesc> opToSamplePruner) {
+ HashMap<TableScanOperator, sampleDesc> opToSamplePruner,
+ SemanticAnalyzer.GlobalLimitCtx globalLimitCtx) {
this.conf = conf;
this.qb = qb;
this.ast = ast;
@@ -166,6 +169,7 @@ public class ParseContext {
this.groupOpToInputTables = groupOpToInputTables;
this.prunedPartitions = prunedPartitions;
this.opToSamplePruner = opToSamplePruner;
+ this.globalLimitCtx = globalLimitCtx;
}
/**
@@ -486,4 +490,12 @@ public class ParseContext {
public void setMapJoinContext(Map<MapJoinOperator, QBJoinTree> mapJoinContext) {
this.mapJoinContext = mapJoinContext;
}
+
+ public SemanticAnalyzer.GlobalLimitCtx getGlobalLimitCtx() {
+ return globalLimitCtx;
+ }
+
+ public void setGlobalLimitCtx(SemanticAnalyzer.GlobalLimitCtx globalLimitCtx) {
+ this.globalLimitCtx = globalLimitCtx;
+ }
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java Tue Apr 19 18:18:20 2011
@@ -480,4 +480,20 @@ public class QBParseInfo {
return tableSpecs.get(tName.next());
}
+ public HashMap<String, Integer> getDestToLimit() {
+ return destToLimit;
+ }
+
+ public LinkedHashMap<String, LinkedHashMap<String, ASTNode>> getDestToAggregationExprs() {
+ return destToAggregationExprs;
+ }
+
+ public HashMap<String, List<ASTNode>> getDestToDistinctFuncExprs() {
+ return destToDistinctFuncExprs;
+ }
+
+ public HashMap<String, TableSample> getNameToSample() {
+ return nameToSample;
+ }
+
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Tue Apr 19 18:18:20 2011
@@ -191,6 +191,50 @@ public class SemanticAnalyzer extends Ba
private CreateViewDesc createVwDesc;
private ASTNode viewSelect;
private final UnparseTranslator unparseTranslator;
+ private final GlobalLimitCtx globalLimitCtx = new GlobalLimitCtx();
+
+ public static class GlobalLimitCtx {
+ private boolean enable = false;
+ private int globalLimit = -1;
+ private boolean hasTransformOrUDTF = false;
+ private LimitDesc lastReduceLimitDesc = null;
+
+ public int getGlobalLimit() {
+ return globalLimit;
+ }
+
+ public boolean ifHasTransformOrUDTF() {
+ return hasTransformOrUDTF;
+ }
+
+ public void setHasTransformOrUDTF(boolean hasTransformOrUDTF) {
+ this.hasTransformOrUDTF = hasTransformOrUDTF;
+ }
+
+ public LimitDesc getLastReduceLimitDesc() {
+ return lastReduceLimitDesc;
+ }
+
+ public void setLastReduceLimitDesc(LimitDesc lastReduceLimitDesc) {
+ this.lastReduceLimitDesc = lastReduceLimitDesc;
+ }
+
+
+ public boolean isEnable() {
+ return enable;
+ }
+
+ public void enableOpt(int globalLimit) {
+ this.enable = true;
+ this.globalLimit = globalLimit;
+ }
+
+ public void disableOpt() {
+ this.enable = false;
+ this.globalLimit = -1;
+ this.lastReduceLimitDesc = null;
+ }
+ }
private static class Phase1Ctx {
String dest;
@@ -262,7 +306,7 @@ public class SemanticAnalyzer extends Ba
topSelOps, opParseCtx, joinContext, topToTable, loadTableWork,
loadFileWork, ctx, idToTableNameMap, destTableId, uCtx,
listMapJoinOpsNoReducer, groupOpToInputTables, prunedPartitions,
- opToSamplePruner);
+ opToSamplePruner, globalLimitCtx);
}
@SuppressWarnings("nls")
@@ -1904,6 +1948,7 @@ public class SemanticAnalyzer extends Ba
boolean isInTransform = (selExprList.getChild(posn).getChild(0).getType() ==
HiveParser.TOK_TRANSFORM);
if (isInTransform) {
+ globalLimitCtx.setHasTransformOrUDTF(true);
trfm = (ASTNode) selExprList.getChild(posn).getChild(0);
}
@@ -1927,6 +1972,9 @@ public class SemanticAnalyzer extends Ba
genericUDTF = fi.getGenericUDTF();
}
isUDTF = (genericUDTF != null);
+ if (isUDTF) {
+ globalLimitCtx.setHasTransformOrUDTF(true);
+ }
if (isUDTF && !fi.isNative()) {
unparseTranslator.addIdentifierTranslation((ASTNode) udtfExpr
.getChild(0));
@@ -3962,8 +4010,12 @@ public class SemanticAnalyzer extends Ba
// Add the limit operator to get the value fields
RowResolver inputRR = opParseCtx.get(input).getRowResolver();
+
+ LimitDesc limitDesc = new LimitDesc(limit);
+ globalLimitCtx.setLastReduceLimitDesc(limitDesc);
+
Operator limitMap = putOpInsertMap(OperatorFactory.getAndMakeChild(
- new LimitDesc(limit), new RowSchema(inputRR.getColumnInfos()), input),
+ limitDesc, new RowSchema(inputRR.getColumnInfos()), input),
inputRR);
if (LOG.isDebugEnabled()) {
@@ -6161,6 +6213,53 @@ public class SemanticAnalyzer extends Ba
}
}
+
+ /**
+ * Recursively check the limit number in all sub queries
+ * @param qbParseInfo
+ * @return if there is one and only one limit for all subqueries, return the limit
+ * if there is no limit, return 0
+ * otherwise, return null
+ */
+ private Integer checkQbpForGlobalLimit(QB localQb) {
+ QBParseInfo qbParseInfo = localQb.getParseInfo();
+ if (localQb.getNumSelDi() == 0 && qbParseInfo.getDestToClusterBy().isEmpty()
+ && qbParseInfo.getDestToDistributeBy().isEmpty()
+ && qbParseInfo.getDestToOrderBy().isEmpty()
+ && qbParseInfo.getDestToSortBy().isEmpty()
+ && qbParseInfo.getDestToAggregationExprs().size() <= 1
+ && qbParseInfo.getDestToDistinctFuncExprs().size() <= 1
+ && qbParseInfo.getNameToSample().isEmpty()) {
+ if ((qbParseInfo.getDestToAggregationExprs().size() < 1 ||
+ qbParseInfo.getDestToAggregationExprs().values().iterator().next().isEmpty()) &&
+ (qbParseInfo.getDestToDistinctFuncExprs().size() < 1 ||
+ qbParseInfo.getDestToDistinctFuncExprs().values().iterator().next().isEmpty())
+ && qbParseInfo.getDestToLimit().size() <= 1) {
+ Integer retValue;
+ if (qbParseInfo.getDestToLimit().size() == 0) {
+ retValue = 0;
+ } else {
+ retValue = qbParseInfo.getDestToLimit().values().iterator().next().intValue();
+ }
+
+ for (String alias : localQb.getSubqAliases()) {
+ Integer limit = checkQbpForGlobalLimit(localQb.getSubqForAlias(alias).getQB());
+ if (limit == null) {
+ return null;
+ } else if (retValue > 0 && limit > 0) {
+ // Any query has more than one LIMITs shown in the query is not
+ // qualified to this optimization
+ return null;
+ } else if (limit > 0) {
+ retValue = limit;
+ }
+ }
+ return retValue;
+ }
+ }
+ return null;
+ }
+
@SuppressWarnings("nls")
private void genMapRedTasks(QB qb) throws SemanticException {
FetchWork fetch = null;
@@ -6253,6 +6352,71 @@ public class SemanticAnalyzer extends Ba
}
}
+ // determine the query qualifies reduce input size for LIMIT
+ // The query only qualifies when there are only one top operator
+ // and there is no transformer or UDTF
+ if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVELIMITOPTENABLE)
+ && ctx.getTryCount() == 0 && topOps.size() == 1
+ && !globalLimitCtx.ifHasTransformOrUDTF()) {
+
+ // Here we recursively check:
+ // 1. whether there are exact one LIMIT in the query
+ // 2. whether there is no aggregation, group-by, distinct, sort by,
+ // distributed by, or table sampling in any of the sub-query.
+ // The query only qualifies if both conditions are satisfied.
+ //
+ // Example qualified queries:
+ // CREATE TABLE ... AS SELECT col1, col2 FROM tbl LIMIT ..
+ // INSERT OVERWRITE TABLE ... SELECT col1, hash(col2), split(col1)
+ // FROM ... LIMIT...
+ // SELECT * FROM (SELECT col1 as col2 (SELECT * FROM ...) t1 LIMIT ...) t2);
+ //
+ Integer tempGlobalLimit = checkQbpForGlobalLimit(qb);
+
+ // query qualify for the optimization
+ if (tempGlobalLimit != null && tempGlobalLimit != 0) {
+ TableScanOperator ts = (TableScanOperator) topOps.values().toArray()[0];
+ Table tab = topToTable.get(ts);
+
+ if (!tab.isPartitioned()) {
+ if (qbParseInfo.getDestToWhereExpr().isEmpty()) {
+ globalLimitCtx.enableOpt(tempGlobalLimit);
+ }
+ } else {
+ // check if the pruner only contains partition columns
+ if (PartitionPruner.onlyContainsPartnCols(tab,
+ opToPartPruner.get(ts))) {
+
+ PrunedPartitionList partsList = null;
+ try {
+ partsList = opToPartList.get(ts);
+ if (partsList == null) {
+ partsList = PartitionPruner.prune(tab,
+ opToPartPruner.get(ts), conf, (String) topOps.keySet()
+ .toArray()[0], prunedPartitions);
+ opToPartList.put(ts, partsList);
+ }
+ } catch (HiveException e) {
+ // Has to use full name to make sure it does not conflict with
+ // org.apache.commons.lang.StringUtils
+ LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e));
+ throw new SemanticException(e.getMessage(), e);
+ }
+
+ // If there is any unknown partition, create a map-reduce job for
+ // the filter to prune correctly
+ if ((partsList.getUnknownPartns().size() == 0)) {
+ globalLimitCtx.enableOpt(tempGlobalLimit);
+ }
+ }
+ }
+ if (globalLimitCtx.isEnable()) {
+ LOG.info("Qualify the optimize that reduces input size for 'limit' for limit "
+ + globalLimitCtx.getGlobalLimit());
+ }
+ }
+ }
+
// In case of a select, use a fetch task instead of a move task
if (qb.getIsQuery()) {
if ((!loadTableWork.isEmpty()) || (loadFileWork.size() != 1)) {
@@ -6269,6 +6433,17 @@ public class SemanticAnalyzer extends Ba
fetchTask = (FetchTask) TaskFactory.get(fetch, conf);
setFetchTask(fetchTask);
+
+ // For the FetchTask, the limit optimiztion requires we fetch all the rows
+ // in memory and count how many rows we get. It's not practical if the
+ // limit factor is too big
+ int fetchLimit = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVELIMITOPTMAXFETCH);
+ if (globalLimitCtx.isEnable() && globalLimitCtx.getGlobalLimit() > fetchLimit) {
+ LOG.info("For FetchTask, LIMIT " + globalLimitCtx.getGlobalLimit() + " > " + fetchLimit
+ + ". Doesn't qualify limit optimiztion.");
+ globalLimitCtx.disableOpt();
+ }
+
} else {
for (LoadTableDesc ltd : loadTableWork) {
Task<MoveWork> tsk = TaskFactory.get(new MoveWork(null, null, ltd, null, false),
@@ -6310,10 +6485,11 @@ public class SemanticAnalyzer extends Ba
}
// generate map reduce plans
+ ParseContext tempParseContext = getParseContext();
GenMRProcContext procCtx = new GenMRProcContext(
conf,
new HashMap<Operator<? extends Serializable>, Task<? extends Serializable>>(),
- new ArrayList<Operator<? extends Serializable>>(), getParseContext(),
+ new ArrayList<Operator<? extends Serializable>>(), tempParseContext,
mvTask, rootTasks,
new LinkedHashMap<Operator<? extends Serializable>, GenMapRedCtx>(),
inputs, outputs);
@@ -6405,6 +6581,21 @@ public class SemanticAnalyzer extends Ba
task.addDependentTask(crtTblTask);
}
}
+
+ if (globalLimitCtx.isEnable() && fetchTask != null) {
+ int fetchLimit = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVELIMITOPTMAXFETCH);
+ LOG.info("set least row check for FetchTask: " + globalLimitCtx.getGlobalLimit());
+ fetchTask.getWork().setLeastNumRows(globalLimitCtx.getGlobalLimit());
+ }
+
+ if (globalLimitCtx.isEnable() && globalLimitCtx.getLastReduceLimitDesc() != null) {
+ LOG.info("set least row check for LimitDesc: " + globalLimitCtx.getGlobalLimit());
+ globalLimitCtx.getLastReduceLimitDesc().setLeastRows(globalLimitCtx.getGlobalLimit());
+ List<ExecDriver> mrTasks = Utilities.getMRTasks(rootTasks);
+ for (ExecDriver tsk : mrTasks) {
+ tsk.setRetryCmdWhenFail(true);
+ }
+ }
}
/**
@@ -6595,6 +6786,7 @@ public class SemanticAnalyzer extends Ba
// continue analyzing from the child ASTNode.
doPhase1(child, qb, initPhase1Ctx());
+
LOG.info("Completed phase 1 of Semantic Analysis");
getMetaData(qb);
@@ -6623,7 +6815,7 @@ public class SemanticAnalyzer extends Ba
opToPartList, topOps, topSelOps, opParseCtx, joinContext, topToTable,
loadTableWork, loadFileWork, ctx, idToTableNameMap, destTableId, uCtx,
listMapJoinOpsNoReducer, groupOpToInputTables, prunedPartitions,
- opToSamplePruner);
+ opToSamplePruner, globalLimitCtx);
Optimizer optm = new Optimizer();
optm.setPctx(pCtx);
@@ -6716,7 +6908,7 @@ public class SemanticAnalyzer extends Ba
throw new SemanticException(
ErrorMsg.VIEW_PARTITION_MISMATCH.getMsg());
}
-
+
// Get the partition columns from the end of derivedSchema.
List<FieldSchema> partitionColumns = derivedSchema.subList(
derivedSchema.size() - partColNames.size(),
@@ -6787,7 +6979,7 @@ public class SemanticAnalyzer extends Ba
TypeCheckCtx tcCtx = new TypeCheckCtx(input);
return genExprNodeDesc(expr, input, tcCtx);
}
-
+
/**
* Generates an expression node descriptor for the expression passed in the
* arguments. This function uses the row resolver and the metadata information
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java Tue Apr 19 18:18:20 2011
@@ -39,6 +39,7 @@ public class FetchWork implements Serial
private ArrayList<PartitionDesc> partDesc;
private int limit;
+ private int leastNumRows;
/**
* Serialization Null Format for the serde used to fetch data.
@@ -189,6 +190,14 @@ public class FetchWork implements Serial
this.limit = limit;
}
+ public int getLeastNumRows() {
+ return leastNumRows;
+ }
+
+ public void setLeastNumRows(int leastNumRows) {
+ this.leastNumRows = leastNumRows;
+ }
+
@Override
public String toString() {
if (tblDir != null) {
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/LimitDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/LimitDesc.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/LimitDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/LimitDesc.java Tue Apr 19 18:18:20 2011
@@ -28,6 +28,7 @@ import java.io.Serializable;
public class LimitDesc implements Serializable {
private static final long serialVersionUID = 1L;
private int limit;
+ private int leastRows = -1;
public LimitDesc() {
}
@@ -44,4 +45,12 @@ public class LimitDesc implements Serial
this.limit = limit;
}
+ public int getLeastRows() {
+ return leastRows;
+ }
+
+ public void setLeastRows(int leastRows) {
+ this.leastRows = leastRows;
+ }
+
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java Tue Apr 19 18:18:20 2011
@@ -18,8 +18,10 @@
package org.apache.hadoop.hive.ql.processors;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+
public interface CommandProcessor {
public void init();
- public CommandProcessorResponse run(String command);
+ public CommandProcessorResponse run(String command) throws CommandNeedRetryException;
}
Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1095153&r1=1095152&r2=1095153&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Tue Apr 19 18:18:20 2011
@@ -626,7 +626,13 @@ public class QTestUtil {
}
public int execute(String tname) {
- return drv.run(qMap.get(tname)).getResponseCode();
+ try {
+ return drv.run(qMap.get(tname)).getResponseCode();
+ } catch (CommandNeedRetryException e) {
+ // TODO Auto-generated catch block
+ e.printStackTrace();
+ return -1;
+ }
}
public int executeClient(String tname) {
Added: hive/trunk/ql/src/test/queries/clientpositive/global_limit.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/global_limit.q?rev=1095153&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/global_limit.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/global_limit.q Tue Apr 19 18:18:20 2011
@@ -0,0 +1,68 @@
+set hive.limit.optimize.enable=true;
+set hive.limit.optimize.limit.file=2;
+
+drop table gl_tgt;
+drop table gl_src1;
+drop table gl_src2;
+drop table gl_src_part1;
+
+
+create table gl_src1 (key int, value string) stored as textfile;
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1;
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1;
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1;
+
+
+
+
+set hive.limit.row.max.size=100;
+
+-- need one file
+create table gl_tgt as select key from gl_src1 limit 1;
+select * from gl_tgt;
+-- need two files
+select 'x', split(value,',') from gl_src1 limit 20;
+-- no sufficient files
+select key, value, split(value,',') from gl_src1 limit 30;
+-- need all files
+select key from gl_src1 limit 100;
+set hive.limit.optimize.limit.file=4;
+select key from gl_src1 limit 30;
+
+-- not qualified cases
+select key, count(1) from gl_src1 group by key order by key limit 5;
+select distinct key from gl_src1 limit 10;
+select count(1) from gl_src1 limit 1;
+select transform(*) using "tr _ \n" as t from
+(select "a_a_a_a_a_a_" from gl_src1 limit 100) subq;
+select key from (select * from (select key,value from gl_src1)t1 limit 10)t2 limit 2000;
+
+-- complicated queries
+select key from (select * from (select key,value from gl_src1 limit 10)t1 )t2;
+select key from (select * from (select key,value from gl_src1)t1 limit 10)t2;
+insert overwrite table gl_tgt select key+1 from (select * from (select key,value from gl_src1)t1)t2 limit 10;
+select * from gl_tgt;
+
+-- empty table
+create table gl_src2 (key int, value string) stored as textfile;
+select key from gl_src2 limit 10;
+
+-- partition
+create table gl_src_part1 (key int, value string) partitioned by (p string) stored as textfile;
+load data local inpath '../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11');
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
+
+select key from gl_src_part1 where p like '1%' limit 10;
+select key from gl_src_part1 where p='11' limit 10;
+select key from gl_src_part1 where p='12' limit 10;
+select key from gl_src_part1 where p='13' limit 10;
+alter table gl_src_part1 add partition (p='13');
+select key from gl_src_part1 where p='13' limit 10;
+select key from gl_src_part1 where p='12' limit 1000;
+
+drop table gl_src1;
+drop table gl_src2;
+drop table gl_src_part1;
+drop table gl_tgt
\ No newline at end of file
Added: hive/trunk/ql/src/test/results/clientpositive/global_limit.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/global_limit.q.out?rev=1095153&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/global_limit.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/global_limit.q.out Tue Apr 19 18:18:20 2011
@@ -0,0 +1,1648 @@
+PREHOOK: query: drop table gl_tgt
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table gl_tgt
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table gl_src1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table gl_src1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table gl_src2
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table gl_src2
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table gl_src_part1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table gl_src_part1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table gl_src1 (key int, value string) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table gl_src1 (key int, value string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@gl_src1
+PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@gl_src1
+POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@gl_src1
+PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@gl_src1
+POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@gl_src1
+PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@gl_src1
+POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@gl_src1
+PREHOOK: query: -- need one file
+create table gl_tgt as select key from gl_src1 limit 1
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@gl_src1
+POSTHOOK: query: -- need one file
+create table gl_tgt as select key from gl_src1 limit 1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: default@gl_tgt
+PREHOOK: query: select * from gl_tgt
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_tgt
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-14_715_6666058695580705176/-mr-10000
+POSTHOOK: query: select * from gl_tgt
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_tgt
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-14_715_6666058695580705176/-mr-10000
+165
+PREHOOK: query: -- need two files
+select 'x', split(value,',') from gl_src1 limit 20
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-14_930_3390626062169619061/-mr-10000
+POSTHOOK: query: -- need two files
+select 'x', split(value,',') from gl_src1 limit 20
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-14_930_3390626062169619061/-mr-10000
+x ["val_165"]
+x ["val_484"]
+x ["val_150"]
+x ["val_224"]
+x ["val_66"]
+x ["val_213"]
+x ["val_374"]
+x ["val_495"]
+x ["val_37"]
+x ["val_327"]
+x ["val_15"]
+x ["val_338"]
+x ["val_459"]
+x ["val_466"]
+x ["val_396"]
+x ["val_309"]
+x ["val_367"]
+x ["val_0"]
+x ["val_455"]
+x ["val_316"]
+PREHOOK: query: -- no sufficient files
+select key, value, split(value,',') from gl_src1 limit 30
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-17_975_1520062346401074551/-mr-10000
+POSTHOOK: query: -- no sufficient files
+select key, value, split(value,',') from gl_src1 limit 30
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-17_975_1520062346401074551/-mr-10000
+165 val_165 ["val_165"]
+484 val_484 ["val_484"]
+150 val_150 ["val_150"]
+224 val_224 ["val_224"]
+66 val_66 ["val_66"]
+213 val_213 ["val_213"]
+374 val_374 ["val_374"]
+495 val_495 ["val_495"]
+37 val_37 ["val_37"]
+327 val_327 ["val_327"]
+15 val_15 ["val_15"]
+338 val_338 ["val_338"]
+459 val_459 ["val_459"]
+466 val_466 ["val_466"]
+396 val_396 ["val_396"]
+309 val_309 ["val_309"]
+367 val_367 ["val_367"]
+0 val_0 ["val_0"]
+455 val_455 ["val_455"]
+316 val_316 ["val_316"]
+345 val_345 ["val_345"]
+129 val_129 ["val_129"]
+378 val_378 ["val_378"]
+4 val_4 ["val_4"]
+356 val_356 ["val_356"]
+169 val_169 ["val_169"]
+125 val_125 ["val_125"]
+437 val_437 ["val_437"]
+286 val_286 ["val_286"]
+187 val_187 ["val_187"]
+PREHOOK: query: -- need all files
+select key from gl_src1 limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-21_006_2990988342025987107/-mr-10000
+POSTHOOK: query: -- need all files
+select key from gl_src1 limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-21_006_2990988342025987107/-mr-10000
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+15
+338
+459
+466
+396
+309
+367
+0
+455
+316
+345
+129
+378
+4
+356
+169
+125
+437
+286
+187
+176
+459
+51
+103
+239
+213
+176
+275
+260
+404
+217
+84
+466
+8
+411
+172
+129
+158
+0
+26
+165
+327
+51
+404
+95
+282
+187
+316
+169
+77
+0
+118
+282
+419
+15
+118
+19
+224
+309
+389
+327
+242
+392
+242
+396
+95
+11
+143
+228
+33
+103
+367
+239
+480
+202
+316
+235
+80
+44
+466
+257
+190
+114
+396
+217
+125
+187
+480
+491
+305
+PREHOOK: query: select key from gl_src1 limit 30
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-24_041_8059705298736859124/-mr-10000
+POSTHOOK: query: select key from gl_src1 limit 30
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-24_041_8059705298736859124/-mr-10000
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+15
+338
+459
+466
+396
+309
+367
+0
+455
+316
+345
+129
+378
+4
+356
+169
+125
+437
+286
+187
+PREHOOK: query: -- not qualified cases
+select key, count(1) from gl_src1 group by key order by key limit 5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-27_042_2819477757997397479/-mr-10000
+POSTHOOK: query: -- not qualified cases
+select key, count(1) from gl_src1 group by key order by key limit 5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-27_042_2819477757997397479/-mr-10000
+0 9
+4 3
+8 3
+11 3
+15 6
+PREHOOK: query: select distinct key from gl_src1 limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-32_925_3439775789836285636/-mr-10000
+POSTHOOK: query: select distinct key from gl_src1 limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-32_925_3439775789836285636/-mr-10000
+0
+4
+8
+11
+15
+19
+26
+33
+37
+44
+PREHOOK: query: select count(1) from gl_src1 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-36_043_608144804096350391/-mr-10000
+POSTHOOK: query: select count(1) from gl_src1 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-36_043_608144804096350391/-mr-10000
+354
+PREHOOK: query: select transform(*) using "tr _ \n" as t from
+(select "a_a_a_a_a_a_" from gl_src1 limit 100) subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-39_121_1333918197839932185/-mr-10000
+POSTHOOK: query: select transform(*) using "tr _ \n" as t from
+(select "a_a_a_a_a_a_" from gl_src1 limit 100) subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-39_121_1333918197839932185/-mr-10000
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+a
+a
+a
+a
+a
+a
+
+PREHOOK: query: select key from (select * from (select key,value from gl_src1)t1 limit 10)t2 limit 2000
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-42_255_4842424370651858121/-mr-10000
+POSTHOOK: query: select key from (select * from (select key,value from gl_src1)t1 limit 10)t2 limit 2000
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-42_255_4842424370651858121/-mr-10000
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+PREHOOK: query: -- complicated queries
+select key from (select * from (select key,value from gl_src1 limit 10)t1 )t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-45_699_2470030147373346756/-mr-10000
+POSTHOOK: query: -- complicated queries
+select key from (select * from (select key,value from gl_src1 limit 10)t1 )t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-45_699_2470030147373346756/-mr-10000
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+PREHOOK: query: select key from (select * from (select key,value from gl_src1)t1 limit 10)t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-48_750_7605887767273501586/-mr-10000
+POSTHOOK: query: select key from (select * from (select key,value from gl_src1)t1 limit 10)t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-48_750_7605887767273501586/-mr-10000
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+PREHOOK: query: insert overwrite table gl_tgt select key+1 from (select * from (select key,value from gl_src1)t1)t2 limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: default@gl_tgt
+POSTHOOK: query: insert overwrite table gl_tgt select key+1 from (select * from (select key,value from gl_src1)t1)t2 limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: default@gl_tgt
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from gl_tgt
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_tgt
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-57_593_9175560296084414291/-mr-10000
+POSTHOOK: query: select * from gl_tgt
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_tgt
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-57_593_9175560296084414291/-mr-10000
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+166
+485
+151
+225
+67
+214
+375
+496
+38
+328
+PREHOOK: query: -- empty table
+create table gl_src2 (key int, value string) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- empty table
+create table gl_src2 (key int, value string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@gl_src2
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select key from gl_src2 limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src2
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-57_846_8904186104857307463/-mr-10000
+POSTHOOK: query: select key from gl_src2 limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src2
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-06-57_846_8904186104857307463/-mr-10000
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: -- partition
+create table gl_src_part1 (key int, value string) partitioned by (p string) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- partition
+create table gl_src_part1 (key int, value string) partitioned by (p string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@gl_src_part1
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11')
+PREHOOK: type: LOAD
+PREHOOK: Output: default@gl_src_part1
+POSTHOOK: query: load data local inpath '../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11')
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@gl_src_part1
+POSTHOOK: Output: default@gl_src_part1@p=11
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12')
+PREHOOK: type: LOAD
+PREHOOK: Output: default@gl_src_part1
+POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12')
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@gl_src_part1
+POSTHOOK: Output: default@gl_src_part1@p=12
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12')
+PREHOOK: type: LOAD
+PREHOOK: Output: default@gl_src_part1@p=12
+POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12')
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@gl_src_part1@p=12
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12')
+PREHOOK: type: LOAD
+PREHOOK: Output: default@gl_src_part1@p=12
+POSTHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12')
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@gl_src_part1@p=12
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select key from gl_src_part1 where p like '1%' limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src_part1@p=11
+PREHOOK: Input: default@gl_src_part1@p=12
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-03_055_6761445866727035783/-mr-10000
+POSTHOOK: query: select key from gl_src_part1 where p like '1%' limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src_part1@p=11
+POSTHOOK: Input: default@gl_src_part1@p=12
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-03_055_6761445866727035783/-mr-10000
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+27
+409
+265
+401
+166
+430
+287
+207
+199
+247
+PREHOOK: query: select key from gl_src_part1 where p='11' limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src_part1@p=11
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-06_080_7097976505609243557/-mr-10000
+POSTHOOK: query: select key from gl_src_part1 where p='11' limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src_part1@p=11
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-06_080_7097976505609243557/-mr-10000
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+27
+409
+265
+401
+166
+430
+287
+207
+199
+247
+PREHOOK: query: select key from gl_src_part1 where p='12' limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src_part1@p=12
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-09_087_9176571562410124649/-mr-10000
+POSTHOOK: query: select key from gl_src_part1 where p='12' limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src_part1@p=12
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-09_087_9176571562410124649/-mr-10000
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+PREHOOK: query: select key from gl_src_part1 where p='13' limit 10
+PREHOOK: type: QUERY
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-12_038_7099507780438686856/-mr-10000
+POSTHOOK: query: select key from gl_src_part1 where p='13' limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-12_038_7099507780438686856/-mr-10000
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: alter table gl_src_part1 add partition (p='13')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@gl_src_part1
+POSTHOOK: query: alter table gl_src_part1 add partition (p='13')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: default@gl_src_part1
+POSTHOOK: Output: default@gl_src_part1@p=13
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select key from gl_src_part1 where p='13' limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src_part1@p=13
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-15_137_3923898954213249104/-mr-10000
+POSTHOOK: query: select key from gl_src_part1 where p='13' limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src_part1@p=13
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-15_137_3923898954213249104/-mr-10000
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select key from gl_src_part1 where p='12' limit 1000
+PREHOOK: type: QUERY
+PREHOOK: Input: default@gl_src_part1@p=12
+PREHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-18_041_1214817963525981880/-mr-10000
+POSTHOOK: query: select key from gl_src_part1 where p='12' limit 1000
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@gl_src_part1@p=12
+POSTHOOK: Output: file:/tmp/sdong/hive_2011-04-18_13-07-18_041_1214817963525981880/-mr-10000
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+15
+338
+459
+466
+396
+309
+367
+0
+455
+316
+345
+129
+378
+4
+356
+169
+125
+437
+286
+187
+176
+459
+51
+103
+239
+213
+176
+275
+260
+404
+217
+84
+466
+8
+411
+172
+129
+158
+0
+26
+165
+327
+51
+404
+95
+282
+187
+316
+169
+77
+0
+118
+282
+419
+15
+118
+19
+224
+309
+389
+327
+242
+392
+242
+396
+95
+11
+143
+228
+33
+103
+367
+239
+480
+202
+316
+235
+80
+44
+466
+257
+190
+114
+396
+217
+125
+187
+480
+491
+305
+444
+169
+323
+480
+136
+172
+462
+26
+462
+341
+183
+84
+37
+448
+194
+477
+169
+400
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+15
+338
+459
+466
+396
+309
+367
+0
+455
+316
+345
+129
+378
+4
+356
+169
+125
+437
+286
+187
+176
+459
+51
+103
+239
+213
+176
+275
+260
+404
+217
+84
+466
+8
+411
+172
+129
+158
+0
+26
+165
+327
+51
+404
+95
+282
+187
+316
+169
+77
+0
+118
+282
+419
+15
+118
+19
+224
+309
+389
+327
+242
+392
+242
+396
+95
+11
+143
+228
+33
+103
+367
+239
+480
+202
+316
+235
+80
+44
+466
+257
+190
+114
+396
+217
+125
+187
+480
+491
+305
+444
+169
+323
+480
+136
+172
+462
+26
+462
+341
+183
+84
+37
+448
+194
+477
+169
+400
+165
+484
+150
+224
+66
+213
+374
+495
+37
+327
+15
+338
+459
+466
+396
+309
+367
+0
+455
+316
+345
+129
+378
+4
+356
+169
+125
+437
+286
+187
+176
+459
+51
+103
+239
+213
+176
+275
+260
+404
+217
+84
+466
+8
+411
+172
+129
+158
+0
+26
+165
+327
+51
+404
+95
+282
+187
+316
+169
+77
+0
+118
+282
+419
+15
+118
+19
+224
+309
+389
+327
+242
+392
+242
+396
+95
+11
+143
+228
+33
+103
+367
+239
+480
+202
+316
+235
+80
+44
+466
+257
+190
+114
+396
+217
+125
+187
+480
+491
+305
+444
+169
+323
+480
+136
+172
+462
+26
+462
+341
+183
+84
+37
+448
+194
+477
+169
+400
+PREHOOK: query: drop table gl_src1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@gl_src1
+PREHOOK: Output: default@gl_src1
+POSTHOOK: query: drop table gl_src1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@gl_src1
+POSTHOOK: Output: default@gl_src1
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: drop table gl_src2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@gl_src2
+PREHOOK: Output: default@gl_src2
+POSTHOOK: query: drop table gl_src2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@gl_src2
+POSTHOOK: Output: default@gl_src2
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: drop table gl_src_part1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@gl_src_part1
+PREHOOK: Output: default@gl_src_part1
+POSTHOOK: query: drop table gl_src_part1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@gl_src_part1
+POSTHOOK: Output: default@gl_src_part1
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: drop table gl_tgt
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@gl_tgt
+PREHOOK: Output: default@gl_tgt
+POSTHOOK: query: drop table gl_tgt
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@gl_tgt
+POSTHOOK: Output: default@gl_tgt
+POSTHOOK: Lineage: gl_tgt.key EXPRESSION [(gl_src1)gl_src1.FieldSchema(name:key, type:int, comment:null), ]