You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by nz...@apache.org on 2010/04/06 07:58:55 UTC
svn commit: r931028 [1/4] - in /hadoop/hive/trunk: ./
ql/src/java/org/apache/hadoop/hive/ql/exec/
ql/src/java/org/apache/hadoop/hive/ql/optimizer/
ql/src/java/org/apache/hadoop/hive/ql/plan/
ql/src/test/queries/clientpositive/ ql/src/test/results/clien...
Author: nzhang
Date: Tue Apr 6 05:58:55 2010
New Revision: 931028
URL: http://svn.apache.org/viewvc?rev=931028&view=rev
Log:
HIVE-1290: sort merge join does not work with bucketizedhiveinputformat (Namit Jain via Ning Zhang)
Added:
hadoop/hive/trunk/ql/src/test/queries/clientpositive/bucketmapjoin6.q
hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin6.q.out
Modified:
hadoop/hive/trunk/CHANGES.txt
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BucketMatcher.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultBucketMatcher.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java
hadoop/hive/trunk/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out
hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin2.q.out
hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin3.q.out
hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin4.q.out
hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
hadoop/hive/trunk/ql/src/test/results/clientpositive/smb_mapjoin_7.q.out
Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Tue Apr 6 05:58:55 2010
@@ -336,6 +336,9 @@ Trunk - Unreleased
HIVE-1253. Fix Date_sub and Date_add in case of daylight saving.
(Bryan Talbot via zshao)
+ HIVE-1290. Sort merge join does not work with bucketizedhiveinputformat.
+ (Namit Jain via Ning Zhang)
+
Release 0.5.0 - Unreleased
INCOMPATIBLE CHANGES
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BucketMatcher.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BucketMatcher.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BucketMatcher.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/BucketMatcher.java Tue Apr 6 05:58:55 2010
@@ -25,10 +25,13 @@ import java.util.List;
import org.apache.hadoop.fs.Path;
public interface BucketMatcher {
-
+
public List<Path> getAliasBucketFiles(String currentInputFile, String refTableAlias, String alias);
-
+
public void setAliasBucketFileNameMapping(
LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketFileNameMapping);
-}
\ No newline at end of file
+ public LinkedHashMap<String, Integer> getBucketFileNameMapping();
+
+ public void setBucketFileNameMapping(LinkedHashMap<String, Integer> bucketFileNameMapping);
+}
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultBucketMatcher.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultBucketMatcher.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultBucketMatcher.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DefaultBucketMatcher.java Tue Apr 6 05:58:55 2010
@@ -28,13 +28,16 @@ import org.apache.commons.logging.LogFac
import org.apache.hadoop.fs.Path;
public class DefaultBucketMatcher implements BucketMatcher {
-
+
protected Log LOG = LogFactory.getLog(this.getClass().getName());
- //MAPPING: bucket_file_name_in_big_tble->{alias_table->corresonding_bucket_file_names}
+ //MAPPING: bucket_file_name_in_big_table->{alias_table->corresonding_bucket_file_names}
private LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketMapping;
-
+
+ private LinkedHashMap<String, Integer> bucketFileNameMapping;
+
public DefaultBucketMatcher(){
+ bucketFileNameMapping = new LinkedHashMap<String, Integer>();
}
public List<Path> getAliasBucketFiles(String refTableInputFile, String refTableAlias, String alias) {
@@ -48,10 +51,18 @@ public class DefaultBucketMatcher implem
}
return paths;
}
-
+
public void setAliasBucketFileNameMapping(
LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketFileNameMapping) {
this.aliasBucketMapping = aliasBucketFileNameMapping;
}
-
+
+ public LinkedHashMap<String, Integer> getBucketFileNameMapping() {
+ return bucketFileNameMapping;
+ }
+
+ public void setBucketFileNameMapping(LinkedHashMap<String, Integer> bucketFileNameMapping) {
+ this.bucketFileNameMapping = bucketFileNameMapping;
+ }
+
}
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java Tue Apr 6 05:58:55 2010
@@ -69,12 +69,14 @@ public class ExecMapper extends MapReduc
private long nextCntr = 1;
private String lastInputFile = null;
private MapredLocalWork localWork = null;
-
+
private ExecMapperContext execContext = new ExecMapperContext();
-
+
public static class ExecMapperContext {
boolean inputFileChanged = false;
String currentInputFile;
+ Integer fileId = new Integer(-1);
+
JobConf jc;
public boolean isInputFileChanged() {
return inputFileChanged;
@@ -94,6 +96,14 @@ public class ExecMapper extends MapReduc
public void setJc(JobConf jc) {
this.jc = jc;
}
+
+ public Integer getFileId() {
+ return fileId;
+ }
+ public void setFileId(Integer fileId) {
+ this.fileId = fileId;
+ }
+
}
@Override
@@ -124,7 +134,7 @@ public class ExecMapper extends MapReduc
mo.setExecContext(execContext);
mo.initializeLocalWork(jc);
mo.initialize(jc, null);
-
+
// initialize map local work
localWork = mrwork.getMapLocalWork();
if (localWork == null) {
@@ -171,7 +181,7 @@ public class ExecMapper extends MapReduc
mo.setOutputCollector(oc);
mo.setReporter(rp);
}
-
+
if(inputFileChanged()) {
if (this.localWork != null
&& (localWork.getInputFileChangeSensitive() || this.lastInputFile == null)) {
@@ -179,7 +189,7 @@ public class ExecMapper extends MapReduc
}
this.lastInputFile = HiveConf.getVar(jc, HiveConf.ConfVars.HADOOPMAPFILENAME);
}
-
+
try {
if (mo.getDone()) {
done = true;
@@ -215,7 +225,7 @@ public class ExecMapper extends MapReduc
* mapper's input file, the work need to clear context and re-initialization
* after the input file changed. This is first introduced to process bucket
* map join.
- *
+ *
* @return
*/
private boolean inputFileChanged() {
@@ -240,12 +250,12 @@ public class ExecMapper extends MapReduc
int fetchOpRows = 0;
String alias = entry.getKey();
FetchOperator fetchOp = entry.getValue();
-
+
if(inputFileChangeSenstive) {
fetchOp.clearFetchContext();
setUpFetchOpContext(fetchOp, alias);
}
-
+
Operator<? extends Serializable> forwardOp = localWork
.getAliasToWork().get(alias);
@@ -283,7 +293,7 @@ public class ExecMapper extends MapReduc
}
}
}
-
+
private void setUpFetchOpContext(FetchOperator fetchOp, String alias)
throws Exception {
String currentInputFile = HiveConf.getVar(jc, HiveConf.ConfVars.HADOOPMAPFILENAME);
@@ -291,13 +301,14 @@ public class ExecMapper extends MapReduc
Class<? extends BucketMatcher> bucketMatcherCls = bucketMatcherCxt.getBucketMatcherClass();
BucketMatcher bucketMatcher = (BucketMatcher) ReflectionUtils.newInstance(bucketMatcherCls, null);
bucketMatcher.setAliasBucketFileNameMapping(bucketMatcherCxt.getAliasBucketFileNameMapping());
+
List<Path> aliasFiles = bucketMatcher.getAliasBucketFiles(currentInputFile,
bucketMatcherCxt.getMapJoinBigTableAlias(),
alias);
Iterator<Path> iter = aliasFiles.iterator();
fetchOp.setupContext(iter, null);
}
-
+
private long getNextCntr(long cntr) {
// A very simple counter to keep track of number of rows processed by the
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java Tue Apr 6 05:58:55 2010
@@ -118,48 +118,18 @@ public class FileSinkOperator extends Te
private void commit(int idx) throws IOException {
if (isNativeTable) {
if (!fs.rename(outPaths[idx], finalPaths[idx])) {
- throw new IOException("Unable to rename output to: "
+ throw new IOException("Unable to rename output to: "
+ finalPaths[idx]);
}
}
LOG.info("Committed " + outPaths[idx] + " to output file: " + finalPaths[idx]);
}
- @Override
- protected void initializeOp(Configuration hconf) throws HiveException {
+ private boolean filesCreated = false;
+ private void openFiles(Configuration hconf) throws HiveException {
+ if (filesCreated)
+ return;
try {
- serializer = (Serializer) conf.getTableInfo().getDeserializerClass()
- .newInstance();
- serializer.initialize(null, conf.getTableInfo().getProperties());
- isNativeTable = !conf.getTableInfo().isNonNative();
-
- JobConf jc;
- if (hconf instanceof JobConf) {
- jc = (JobConf) hconf;
- } else {
- // test code path
- jc = new JobConf(hconf, ExecDriver.class);
- }
-
- multiFileSpray = conf.isMultiFileSpray();
- totalFiles = conf.getTotalFiles();
- numFiles = conf.getNumFiles();
-
- if (multiFileSpray) {
- partitionEval = new ExprNodeEvaluator[conf.getPartitionCols().size()];
- int i = 0;
- for (ExprNodeDesc e : conf.getPartitionCols()) {
- partitionEval[i++] = ExprNodeEvaluatorFactory.get(e);
- }
-
- partitionObjectInspectors = initEvaluators(partitionEval, outputObjInspector);
- prtner = (HivePartitioner<HiveKey, Object>)ReflectionUtils.newInstance(jc.getPartitionerClass(), null);
- }
-
- outWriters = new RecordWriter[numFiles];
- outPaths = new Path[numFiles];
- finalPaths = new Path[numFiles];
-
String specPath = conf.getDirName();
Path tmpPath = Utilities.toTempPath(specPath);
Set<Integer> seenBuckets = new HashSet<Integer>();
@@ -175,6 +145,17 @@ public class FileSinkOperator extends Te
for (int idx = 0; idx < totalFiles; idx++) {
String taskId = Utilities.getTaskId(hconf);
+ if (this.getExecContext() != null && this.getExecContext().getFileId() != -1) {
+ LOG.info("replace taskId from execContext ");
+
+ taskId = Utilities.replaceTaskIdFromFilename(taskId, this.getExecContext().getFileId());
+
+ LOG.info("new taskId: FS " + taskId);
+
+ assert !multiFileSpray;
+ assert totalFiles == 1;
+ }
+
if (multiFileSpray) {
key.setHashCode(idx);
@@ -227,9 +208,57 @@ public class FileSinkOperator extends Te
// in recent hadoop versions, use deleteOnExit to clean tmp files.
if (isNativeTable) {
autoDelete = ShimLoader.getHadoopShims().fileSystemDeleteOnExit(fs,
- outPaths[0]);
+ outPaths[0]);
+ }
+ } catch (HiveException e) {
+ throw e;
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HiveException(e);
+ }
+
+ filesCreated = true;
+ }
+
+ private Configuration hconf;
+ private JobConf jc;
+
+ @Override
+ protected void initializeOp(Configuration hconf) throws HiveException {
+ try {
+ filesCreated = false;
+ this.hconf = hconf;
+ serializer = (Serializer) conf.getTableInfo().getDeserializerClass()
+ .newInstance();
+ serializer.initialize(null, conf.getTableInfo().getProperties());
+ isNativeTable = !conf.getTableInfo().isNonNative();
+
+ if (hconf instanceof JobConf) {
+ jc = (JobConf) hconf;
+ } else {
+ // test code path
+ jc = new JobConf(hconf, ExecDriver.class);
}
+ multiFileSpray = conf.isMultiFileSpray();
+ totalFiles = conf.getTotalFiles();
+ numFiles = conf.getNumFiles();
+
+ if (multiFileSpray) {
+ partitionEval = new ExprNodeEvaluator[conf.getPartitionCols().size()];
+ int i = 0;
+ for (ExprNodeDesc e : conf.getPartitionCols()) {
+ partitionEval[i++] = ExprNodeEvaluatorFactory.get(e);
+ }
+
+ partitionObjectInspectors = initEvaluators(partitionEval, outputObjInspector);
+ prtner = (HivePartitioner<HiveKey, Object>)ReflectionUtils.newInstance(jc.getPartitionerClass(), null);
+ }
+
+ outWriters = new RecordWriter[numFiles];
+ outPaths = new Path[numFiles];
+ finalPaths = new Path[numFiles];
+
int id = conf.getDestTableId();
if ((id != 0) && (id <= TableIdEnum.values().length)) {
String enumName = "TABLE_ID_" + String.valueOf(id) + "_ROWCOUNT";
@@ -251,6 +280,9 @@ public class FileSinkOperator extends Te
@Override
public void processOp(Object row, int tag) throws HiveException {
+ if (!filesCreated)
+ openFiles(hconf);
+
// Since File Sink is a terminal operator, forward is not called - so,
// maintain the number of output rows explicitly
if (counterNameToEnum != null) {
@@ -295,6 +327,8 @@ public class FileSinkOperator extends Te
@Override
public void closeOp(boolean abort) throws HiveException {
+ if (!filesCreated)
+ openFiles(hconf);
if (!abort) {
for (int idx = 0; idx < numFiles; idx++) {
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java Tue Apr 6 05:58:55 2010
@@ -61,7 +61,7 @@ public class SMBMapJoinOperator extends
transient Map<Byte, ArrayList<Object>> nextKeyWritables;
HashMap<Byte, RowContainer<ArrayList<Object>>> nextGroupStorage;
HashMap<Byte, RowContainer<ArrayList<Object>>> candidateStorage;
-
+
transient HashMap<Byte, String> tagToAlias;
private transient HashMap<Byte, Boolean> fetchOpDone = new HashMap<Byte, Boolean>();
private transient HashMap<Byte, Boolean> foundNextKeyGroup = new HashMap<Byte, Boolean>();
@@ -70,19 +70,19 @@ public class SMBMapJoinOperator extends
public SMBMapJoinOperator() {
}
-
+
public SMBMapJoinOperator(AbstractMapJoinOperator<? extends MapJoinDesc> mapJoinOp) {
super(mapJoinOp);
}
-
+
@Override
protected void initializeOp(Configuration hconf) throws HiveException {
super.initializeOp(hconf);
-
+
firstRow = true;
-
+
closeCalled = false;
-
+
this.firstFetchHappened = false;
nextGroupStorage = new HashMap<Byte, RowContainer<ArrayList<Object>>>();
@@ -101,7 +101,7 @@ public class SMBMapJoinOperator extends
tagToAlias = conf.getTagToAlias();
keyWritables = new HashMap<Byte, ArrayList<Object>>();
nextKeyWritables = new HashMap<Byte, ArrayList<Object>>();
-
+
for (Byte alias : order) {
if(alias != (byte) posBigTable) {
fetchOpDone.put(alias, Boolean.FALSE);;
@@ -109,7 +109,7 @@ public class SMBMapJoinOperator extends
foundNextKeyGroup.put(alias, Boolean.FALSE);
}
}
-
+
@Override
public void initializeLocalWork(Configuration hconf) throws HiveException {
initializeMapredLocalWork(this.getConf(), hconf, this.getConf().getLocalWork(), LOG);
@@ -133,7 +133,7 @@ public class SMBMapJoinOperator extends
l4j.info("fetchoperator for " + entry.getKey() + " created");
}
}
-
+
for (Map.Entry<String, FetchOperator> entry : fetchOperators.entrySet()) {
Operator<? extends Serializable> forwardOp = localWork.getAliasToWork()
.get(entry.getKey());
@@ -144,14 +144,14 @@ public class SMBMapJoinOperator extends
l4j.info("fetchoperator for " + entry.getKey() + " initialized");
}
}
-
+
@Override
public void processOp(Object row, int tag) throws HiveException {
if (this.getExecContext().inputFileChanged) {
if(firstFetchHappened) {
- //we need to first join and flush out data left by the previous file.
- joinFinalLeftData();
+ //we need to first join and flush out data left by the previous file.
+ joinFinalLeftData();
}
//set up the fetch operator for the new input file.
for (Map.Entry<String, FetchOperator> entry : fetchOperators.entrySet()) {
@@ -163,13 +163,13 @@ public class SMBMapJoinOperator extends
this.getExecContext().inputFileChanged = false;
firstFetchHappened = false;
}
-
+
if (!firstFetchHappened) {
firstFetchHappened = true;
// fetch the first group for all small table aliases
for (Byte t : order) {
if(t != (byte)posBigTable) {
- fetchNextGroup(t);
+ fetchNextGroup(t);
}
}
}
@@ -191,7 +191,7 @@ public class SMBMapJoinOperator extends
return;
}
}
-
+
reportProgress();
// the big table has reached a new key group. try to let the small tables
@@ -204,10 +204,10 @@ public class SMBMapJoinOperator extends
//jump out the loop if we need input from the big table
} while (smallestPos != null && smallestPos.size() > 0
&& !smallestPos.contains((byte)this.posBigTable));
-
+
return;
}
-
+
assert !nextKeyGroup;
candidateStorage.get((byte) tag).add(value);
}
@@ -219,7 +219,7 @@ public class SMBMapJoinOperator extends
*/
private void joinFinalLeftData() throws HiveException {
RowContainer bigTblRowContainer = this.candidateStorage.get((byte)this.posBigTable);
-
+
boolean allFetchOpDone = allFetchOpDone();
// if all left data in small tables are less than and equal to the left data
// in big table, let's them catch up
@@ -229,11 +229,11 @@ public class SMBMapJoinOperator extends
bigTblRowContainer = this.candidateStorage.get((byte)this.posBigTable);
allFetchOpDone = allFetchOpDone();
}
-
+
if (allFetchOpDone
&& this.candidateStorage.get((byte) this.posBigTable).size() > 0) {
// if all fetch operator for small tables are done and there are data left
- // in big table
+ // in big table
for (byte t : order) {
if(this.foundNextKeyGroup.get(t) && this.nextKeyWritables.get(t) != null) {
promoteNextGroupToCandidate(t);
@@ -246,9 +246,9 @@ public class SMBMapJoinOperator extends
if (ret == null || ret.size() == 0) {
break;
}
-
+
reportProgress();
-
+
allFetchOpDone = allFetchOpDone();
}
//one final table left
@@ -322,7 +322,7 @@ public class SMBMapJoinOperator extends
}
return needFetchList;
}
-
+
private void fetchNextGroup(Byte t) throws HiveException {
if (foundNextKeyGroup.get(t)) {
// first promote the next group to be the current group if we reached a
@@ -340,7 +340,7 @@ public class SMBMapJoinOperator extends
if(t == (byte)posBigTable) {
return;
}
-
+
//for tables other than the big table, we need to fetch more data until reach a new group or done.
while (!foundNextKeyGroup.get(t)) {
if (fetchOpDone.get(t)) {
@@ -361,7 +361,7 @@ public class SMBMapJoinOperator extends
this.candidateStorage.put(t, this.nextGroupStorage.get(t));
this.nextGroupStorage.put(t, oldRowContainer);
}
-
+
private int compareKeys (ArrayList<Object> k1, ArrayList<Object> k2) {
int ret = 0;
for (int i = 0; i < k1.size() && i < k1.size(); i++) {
@@ -433,6 +433,9 @@ public class SMBMapJoinOperator extends
.getBucketMatcherClass();
BucketMatcher bucketMatcher = (BucketMatcher) ReflectionUtils.newInstance(
bucketMatcherCls, null);
+ this.getExecContext().setFileId(bucketMatcherCxt.getBucketFileNameMapping().get(currentInputFile));
+ LOG.info("set task id: " + this.getExecContext().getFileId());
+
bucketMatcher.setAliasBucketFileNameMapping(bucketMatcherCxt
.getAliasBucketFileNameMapping());
List<Path> aliasFiles = bucketMatcher.getAliasBucketFiles(currentInputFile,
@@ -445,7 +448,7 @@ public class SMBMapJoinOperator extends
if (fetchOperators != null) {
String tble = this.tagToAlias.get(tag);
FetchOperator fetchOp = fetchOperators.get(tble);
-
+
Operator<? extends Serializable> forwardOp = localWork.getAliasToWork()
.get(tble);
try {
@@ -470,7 +473,7 @@ public class SMBMapJoinOperator extends
}
}
}
-
+
transient boolean closeCalled = false;
@Override
public void closeOp(boolean abort) throws HiveException {
@@ -478,7 +481,7 @@ public class SMBMapJoinOperator extends
return;
}
closeCalled = true;
-
+
if ((this.getExecContext() != null && this.getExecContext().inputFileChanged)
|| !firstFetchHappened) {
//set up the fetch operator for the new input file.
@@ -492,23 +495,23 @@ public class SMBMapJoinOperator extends
firstFetchHappened = true;
for (Byte t : order) {
if(t != (byte)posBigTable) {
- fetchNextGroup(t);
+ fetchNextGroup(t);
}
}
}
-
+
joinFinalLeftData();
-
- //clean up
+
+ //clean up
for (Byte alias : order) {
if(alias != (byte) posBigTable) {
fetchOpDone.put(alias, Boolean.FALSE);;
}
foundNextKeyGroup.put(alias, Boolean.FALSE);
}
-
+
localWorkInited = false;
-
+
super.closeOp(abort);
if (fetchOperators != null) {
for (Map.Entry<String, FetchOperator> entry : fetchOperators.entrySet()) {
@@ -518,14 +521,14 @@ public class SMBMapJoinOperator extends
}
}
}
-
+
protected boolean allInitializedParentsAreClosed() {
return true;
}
/**
* Implements the getName function for the Node Interface.
- *
+ *
* @return the name of the operator
*/
@Override
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Tue Apr 6 05:58:55 2010
@@ -225,7 +225,7 @@ public final class Utilities {
}
}
}
-
+
public static class SetDelegate extends DefaultPersistenceDelegate {
@Override
protected Expression instantiate(Object oldInstance, Encoder out) {
@@ -248,9 +248,9 @@ public final class Utilities {
out.writeStatement(new Statement(oldInstance, "add", new Object[]{i.next()}));
}
}
-
+
}
-
+
public static class ListDelegate extends DefaultPersistenceDelegate {
@Override
protected Expression instantiate(Object oldInstance, Encoder out) {
@@ -273,9 +273,9 @@ public final class Utilities {
out.writeStatement(new Statement(oldInstance, "add", new Object[]{i.next()}));
}
}
-
+
}
-
+
public static void setMapRedWork(Configuration job, MapredWork w) {
try {
// use the default file system of the job
@@ -355,7 +355,7 @@ public final class Utilities {
}
}
}
-
+
/**
* Serialize the whole query plan.
*/
@@ -372,10 +372,10 @@ public final class Utilities {
e.setPersistenceDelegate(GroupByDesc.Mode.class, new EnumDelegate());
e.setPersistenceDelegate(Operator.ProgressCounter.class,
new EnumDelegate());
-
+
e.setPersistenceDelegate(org.datanucleus.sco.backed.Map.class, new MapDelegate());
e.setPersistenceDelegate(org.datanucleus.sco.backed.List.class, new ListDelegate());
-
+
e.writeObject(plan);
e.close();
}
@@ -390,9 +390,9 @@ public final class Utilities {
d.close();
return (ret);
}
-
+
/**
- * Serialize the mapredWork object to an output stream. DO NOT use this to
+ * Serialize the mapredWork object to an output stream. DO NOT use this to
* write to standard output since it closes the output stream.
* DO USE mapredWork.toXML() instead.
*/
@@ -681,7 +681,7 @@ public final class Utilities {
/**
* Convert an output stream to a compressed output stream based on codecs and
* compression options specified in the Job Configuration.
- *
+ *
* @param jc
* Job Configuration
* @param out
@@ -698,7 +698,7 @@ public final class Utilities {
* Convert an output stream to a compressed output stream based on codecs
* codecs in the Job Configuration. Caller specifies directly whether file is
* compressed or not
- *
+ *
* @param jc
* Job Configuration
* @param out
@@ -723,7 +723,7 @@ public final class Utilities {
/**
* Based on compression option and configured output codec - get extension for
* output file. This is only required for text files - not sequencefiles
- *
+ *
* @param jc
* Job Configuration
* @param isCompressed
@@ -744,7 +744,7 @@ public final class Utilities {
/**
* Create a sequencefile output stream based on job configuration.
- *
+ *
* @param jc
* Job configuration
* @param fs
@@ -768,7 +768,7 @@ public final class Utilities {
* Create a sequencefile output stream based on job configuration Uses user
* supplied compression flag (rather than obtaining it from the Job
* Configuration).
- *
+ *
* @param jc
* Job configuration
* @param fs
@@ -801,7 +801,7 @@ public final class Utilities {
/**
* Create a RCFile output stream based on job configuration Uses user supplied
* compression flag (rather than obtaining it from the Job Configuration).
- *
+ *
* @param jc
* Job configuration
* @param fs
@@ -903,7 +903,7 @@ public final class Utilities {
* Rename src to dst, or in the case dst already exists, move files in src to
* dst. If there is an existing file with the same name, the new file's name
* will be appended with "_1", "_2", etc.
- *
+ *
* @param fs
* the FileSystem where src and dst are on.
* @param src
@@ -923,7 +923,7 @@ public final class Utilities {
* Rename src to dst, or in the case dst already exists, move files in src to
* dst. If there is an existing file with the same name, the new file's name
* will be appended with "_1", "_2", etc.
- *
+ *
* @param fs
* the FileSystem where src and dst are on.
* @param src
@@ -997,10 +997,25 @@ public final class Utilities {
for (int i = 0; i < taskIdLen - bucketNumLen; i++) {
s.append("0");
}
- String newTaskId = s.toString() + strBucketNum;
- return filename.replaceAll(taskId, newTaskId);
+ String newTaskId = s.toString() + strBucketNum;
+ String[] spl = filename.split(taskId);
+
+ if ((spl.length == 0) || (spl.length == 1)) {
+ return filename.replaceAll(taskId, newTaskId);
+ }
+
+ StringBuffer snew = new StringBuffer();
+ for (int idx = 0; idx < spl.length-1; idx++) {
+ if (idx > 0) {
+ snew.append(taskId);
+ }
+ snew.append(spl[idx]);
+ }
+ snew.append(newTaskId);
+ snew.append(spl[spl.length-1]);
+ return snew.toString();
}
-
+
/**
* Remove all temporary files and duplicate (double-committed) files from a
* given directory.
@@ -1046,7 +1061,7 @@ public final class Utilities {
/**
* Add new elements to the classpath.
- *
+ *
* @param newPaths
* Array of classpath elements
*/
@@ -1079,7 +1094,7 @@ public final class Utilities {
/**
* remove elements from the classpath.
- *
+ *
* @param pathsToRemove
* Array of classpath elements
*/
@@ -1180,7 +1195,7 @@ public final class Utilities {
/**
* Gets the default notification interval to send progress updates to the
* tracker. Useful for operators that may not output data for a while.
- *
+ *
* @param hconf
* @return the interval in milliseconds
*/
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java Tue Apr 6 05:58:55 2010
@@ -67,7 +67,7 @@ import org.apache.hadoop.hive.ql.udf.gen
*this transformation does bucket map join optimization.
*/
public class BucketMapJoinOptimizer implements Transform {
-
+
private static final Log LOG = LogFactory.getLog(GroupByOptimizer.class
.getName());
@@ -130,11 +130,11 @@ public class BucketMapJoinOptimizer impl
}
};
}
-
+
class BucketMapjoinOptProc implements NodeProcessor {
-
+
protected ParseContext pGraphContext;
-
+
public BucketMapjoinOptProc(ParseContext pGraphContext) {
super();
this.pGraphContext = pGraphContext;
@@ -148,11 +148,11 @@ public class BucketMapJoinOptimizer impl
if(context.getListOfRejectedMapjoins().contains(mapJoinOp))
return null;
-
+
QBJoinTree joinCxt = this.pGraphContext.getMapJoinContext().get(mapJoinOp);
if(joinCxt == null)
return null;
-
+
List<String> joinAliases = new ArrayList<String>();
String[] srcs = joinCxt.getBaseSrc();
String[] left = joinCxt.getLeftAliases();
@@ -174,7 +174,7 @@ public class BucketMapJoinOptimizer impl
}
}
}
-
+
MapJoinDesc mjDecs = mapJoinOp.getConf();
LinkedHashMap<String, Integer> aliasToBucketNumberMapping = new LinkedHashMap<String, Integer>();
LinkedHashMap<String, List<String>> aliasToBucketFileNamesMapping = new LinkedHashMap<String, List<String>>();
@@ -183,12 +183,12 @@ public class BucketMapJoinOptimizer impl
// with only one partition presents in each join source tables.
Map<String, Operator<? extends Serializable>> topOps = this.pGraphContext.getTopOps();
Map<TableScanOperator, Table> topToTable = this.pGraphContext.getTopToTable();
-
+
// (partition to bucket file names) and (partition to bucket number) for
// the big table;
LinkedHashMap<Partition, List<String>> bigTblPartsToBucketFileNames = new LinkedHashMap<Partition, List<String>>();
LinkedHashMap<Partition, Integer> bigTblPartsToBucketNumber = new LinkedHashMap<Partition, Integer>();
-
+
for (int index = 0; index < joinAliases.size(); index++) {
String alias = joinAliases.get(index);
TableScanOperator tso = (TableScanOperator) topOps.get(alias);
@@ -208,9 +208,9 @@ public class BucketMapJoinOptimizer impl
}
int partNumber = prunedParts.getConfirmedPartns().size()
+ prunedParts.getUnknownPartns().size();
-
+
if (partNumber > 1) {
- // only allow one partition for small tables
+ // only allow one partition for small tables
if(alias != baseBigAlias) {
return null;
}
@@ -243,7 +243,7 @@ public class BucketMapJoinOptimizer impl
// not contain mappings for the big table. Instead, the mappings are
// contained in bigTblPartsToBucketFileNames and
// bigTblPartsToBucketNumber
-
+
} else {
Partition part = null;
Iterator<Partition> iter = prunedParts.getConfirmedPartns()
@@ -286,7 +286,7 @@ public class BucketMapJoinOptimizer impl
aliasToBucketFileNamesMapping.put(alias, fileNames);
}
}
-
+
// All tables or partitions are bucketed, and their bucket number is
// stored in 'bucketNumbers', we need to check if the number of buckets in
// the big table can be divided by no of buckets in small tables.
@@ -307,24 +307,24 @@ public class BucketMapJoinOptimizer impl
return null;
}
}
-
+
MapJoinDesc desc = mapJoinOp.getConf();
-
- LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketFileNameMapping =
+
+ LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketFileNameMapping =
new LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>>();
-
- //sort bucket names for the big table
+
+ //sort bucket names for the big table
if(bigTblPartsToBucketNumber.size() > 0) {
Collection<List<String>> bucketNamesAllParts = bigTblPartsToBucketFileNames.values();
for(List<String> partBucketNames : bucketNamesAllParts) {
Collections.sort(partBucketNames);
}
} else {
- Collections.sort(aliasToBucketFileNamesMapping.get(baseBigAlias));
+ Collections.sort(aliasToBucketFileNamesMapping.get(baseBigAlias));
}
-
+
// go through all small tables and get the mapping from bucket file name
- // in the big table to bucket file names in small tables.
+ // in the big table to bucket file names in small tables.
for (int j = 0; j < joinAliases.size(); j++) {
String alias = joinAliases.get(j);
if(alias.equals(baseBigAlias))
@@ -332,7 +332,7 @@ public class BucketMapJoinOptimizer impl
Collections.sort(aliasToBucketFileNamesMapping.get(alias));
LinkedHashMap<String, ArrayList<String>> mapping = new LinkedHashMap<String, ArrayList<String>>();
aliasBucketFileNameMapping.put(alias, mapping);
-
+
// for each bucket file in big table, get the corresponding bucket file
// name in the small table.
if (bigTblPartsToBucketNumber.size() > 0) {
@@ -347,14 +347,14 @@ public class BucketMapJoinOptimizer impl
List<String> bigTblBucketNameList = bigTblPartToBucketNames.next().getValue();
fillMapping(baseBigAlias, aliasToBucketNumberMapping,
aliasToBucketFileNamesMapping, alias, mapping, bigTblBucketNum,
- bigTblBucketNameList);
+ bigTblBucketNameList, desc.getBucketFileNameMapping());
}
} else {
List<String> bigTblBucketNameList = aliasToBucketFileNamesMapping.get(baseBigAlias);
int bigTblBucketNum = aliasToBucketNumberMapping.get(baseBigAlias);
fillMapping(baseBigAlias, aliasToBucketNumberMapping,
aliasToBucketFileNamesMapping, alias, mapping, bigTblBucketNum,
- bigTblBucketNameList);
+ bigTblBucketNameList, desc.getBucketFileNameMapping());
}
}
desc.setAliasBucketFileNameMapping(aliasBucketFileNameMapping);
@@ -366,7 +366,8 @@ public class BucketMapJoinOptimizer impl
LinkedHashMap<String, Integer> aliasToBucketNumberMapping,
LinkedHashMap<String, List<String>> aliasToBucketFileNamesMapping,
String alias, LinkedHashMap<String, ArrayList<String>> mapping,
- int bigTblBucketNum, List<String> bigTblBucketNameList) {
+ int bigTblBucketNum, List<String> bigTblBucketNameList,
+ LinkedHashMap<String, Integer> bucketFileNameMapping) {
for (int index = 0; index < bigTblBucketNameList.size(); index++) {
String inputBigTBLBucket = bigTblBucketNameList.get(index);
int smallTblBucketNum = aliasToBucketNumberMapping.get(alias);
@@ -389,6 +390,7 @@ public class BucketMapJoinOptimizer impl
}
}
mapping.put(inputBigTBLBucket, resultFileNames);
+ bucketFileNameMapping.put(inputBigTBLBucket, index);
}
}
@@ -423,12 +425,12 @@ public class BucketMapJoinOptimizer impl
}
return fileNames;
}
-
+
private boolean checkBucketColumns(List<String> bucketColumns, MapJoinDesc mjDesc, int index) {
List<ExprNodeDesc> keys = mjDesc.getKeys().get((byte)index);
if (keys == null || bucketColumns == null || bucketColumns.size() == 0)
return false;
-
+
//get all join columns from join keys stored in MapJoinDesc
List<String> joinCols = new ArrayList<String>();
List<ExprNodeDesc> joinKeys = new ArrayList<ExprNodeDesc>();
@@ -450,30 +452,30 @@ public class BucketMapJoinOptimizer impl
}
// to see if the join columns from a table is exactly this same as its
- // bucket columns
+ // bucket columns
if (joinCols.size() == 0 || joinCols.size() != bucketColumns.size()) {
return false;
}
-
+
for (String col : joinCols) {
if (!bucketColumns.contains(col))
return false;
}
-
+
return true;
}
-
+
}
-
+
class BucketMapjoinOptProcCtx implements NodeProcessorCtx {
// we only convert map joins that follows a root table scan in the same
// mapper. That means there is no reducer between the root table scan and
// mapjoin.
Set<MapJoinOperator> listOfRejectedMapjoins = new HashSet<MapJoinOperator>();
-
+
public Set<MapJoinOperator> getListOfRejectedMapjoins() {
return listOfRejectedMapjoins;
}
-
+
}
}
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java Tue Apr 6 05:58:55 2010
@@ -272,6 +272,7 @@ public final class GenMapRedUtils {
BucketMapJoinContext bucketMJCxt = new BucketMapJoinContext();
localPlan.setBucketMapjoinContext(bucketMJCxt);
bucketMJCxt.setAliasBucketFileNameMapping(aliasBucketFileNameMapping);
+ bucketMJCxt.setBucketFileNameMapping(currMapJoinOp.getConf().getBucketFileNameMapping());
localPlan.setInputFileChangeSensitive(true);
bucketMJCxt.setMapJoinBigTableAlias(currMapJoinOp.getConf().getBigTableAlias());
bucketMJCxt.setBucketMatcherClass(org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class);
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java Tue Apr 6 05:58:55 2010
@@ -30,7 +30,7 @@ import java.util.Map.Entry;
/**
* Map Join operator Descriptor implementation.
- *
+ *
*/
@Explain(displayName = "Common Join Operator")
public class MapJoinDesc extends JoinDesc implements Serializable {
@@ -43,14 +43,16 @@ public class MapJoinDesc extends JoinDes
private int posBigTable;
private Map<Byte, List<Integer>> retainList;
-
+
private transient String bigTableAlias;
-
+
private LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketFileNameMapping;
+ private LinkedHashMap<String, Integer> bucketFileNameMapping;
public MapJoinDesc() {
+ bucketFileNameMapping = new LinkedHashMap<String, Integer>();
}
-
+
public MapJoinDesc(MapJoinDesc clone) {
super(clone);
this.keys = clone.keys;
@@ -60,6 +62,7 @@ public class MapJoinDesc extends JoinDes
this.retainList = clone.retainList;
this.bigTableAlias = clone.bigTableAlias;
this.aliasBucketFileNameMapping = clone.aliasBucketFileNameMapping;
+ this.bucketFileNameMapping = clone.bucketFileNameMapping;
}
public MapJoinDesc(final Map<Byte, List<ExprNodeDesc>> keys,
@@ -71,6 +74,7 @@ public class MapJoinDesc extends JoinDes
this.keyTblDesc = keyTblDesc;
this.valueTblDescs = valueTblDescs;
this.posBigTable = posBigTable;
+ this.bucketFileNameMapping = new LinkedHashMap<String, Integer>();
initRetainExprList();
}
@@ -180,4 +184,12 @@ public class MapJoinDesc extends JoinDes
LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketFileNameMapping) {
this.aliasBucketFileNameMapping = aliasBucketFileNameMapping;
}
+
+ public LinkedHashMap<String, Integer> getBucketFileNameMapping() {
+ return bucketFileNameMapping;
+ }
+
+ public void setBucketFileNameMapping(LinkedHashMap<String, Integer> bucketFileNameMapping) {
+ this.bucketFileNameMapping = bucketFileNameMapping;
+ }
}
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredLocalWork.java Tue Apr 6 05:58:55 2010
@@ -119,6 +119,7 @@ public class MapredLocalWork implements
private Class<? extends BucketMatcher> bucketMatcherClass;
private LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketBaseFileNameMapping;
+ private LinkedHashMap<String, Integer> bucketFileNameMapping;
public void setMapJoinBigTableAlias(String bigTableAlias) {
this.mapJoinBigTableAlias = bigTableAlias;
@@ -214,5 +215,15 @@ public class MapredLocalWork implements
LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketBaseFileNameMapping) {
this.aliasBucketBaseFileNameMapping = aliasBucketBaseFileNameMapping;
}
+
+ @Explain(displayName = "Alias Bucket Output File Name Mapping", normalExplain = false)
+ public LinkedHashMap<String, Integer> getBucketFileNameMapping() {
+ return bucketFileNameMapping;
+ }
+
+ public void setBucketFileNameMapping(LinkedHashMap<String, Integer> bucketFileNameMapping) {
+ this.bucketFileNameMapping = bucketFileNameMapping;
+ }
+
}
}
Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/bucketmapjoin6.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/bucketmapjoin6.q?rev=931028&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/bucketmapjoin6.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/bucketmapjoin6.q Tue Apr 6 05:58:55 2010
@@ -0,0 +1,32 @@
+drop table tmp1;
+create table tmp1 (a string, b string) clustered by (a) sorted by (a) into 10 buckets;
+
+drop table tmp2;
+create table tmp2 (a string, b string) clustered by (a) sorted by (a) into 10 buckets;
+
+
+set hive.enforce.bucketing = true;
+set hive.enforce.sorting = true;
+
+
+insert overwrite table tmp1 select * from src where key < 50;
+insert overwrite table tmp2 select * from src where key < 50;
+
+set hive.optimize.bucketmapjoin = true;
+set hive.optimize.bucketmapjoin.sortedmerge = true;
+set hive.merge.mapfiles=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
+
+drop table tmp3;
+create table tmp3 (a string, b string, c string) clustered by (a) sorted by (a) into 10 buckets;
+
+
+insert overwrite table tmp3
+ select /*+ MAPJOIN(l) */ i.a, i.b, l.b
+ from tmp1 i join tmp2 l ON i.a = l.a;
+
+select * from tmp3;
+
+drop table tmp1;
+drop table tmp2;
+drop table tmp3;
Modified: hadoop/hive/trunk/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/smb_mapjoin_7.q?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/smb_mapjoin_7.q (original)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/smb_mapjoin_7.q Tue Apr 6 05:58:55 2010
@@ -30,7 +30,7 @@ select /*+mapjoin(b)*/ * from smb_bucket
insert overwrite table smb_join_results_empty_bigtable
select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key;
-select * from smb_join_results_empty_bigtable order by k1;
+select * from smb_join_results_empty_bigtable order by k1, v1, k2, v2;
explain
insert overwrite table smb_join_results
@@ -39,7 +39,7 @@ select /*+mapjoin(a)*/ * from smb_bucket
insert overwrite table smb_join_results
select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key;
-select * from smb_join_results order by k1;
+select * from smb_join_results order by k1, v1, k2, v2;
insert overwrite table normal_join_results select * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key;
Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out?rev=931028&r1=931027&r2=931028&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out Tue Apr 6 05:58:55 2010
@@ -161,7 +161,7 @@ STAGE PLANS:
File Output Operator
compressed: false
GlobalTableId: 1
- directory: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10002
+ directory: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10002
NumFilesPerFileSink: 1
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -172,12 +172,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
TotalFiles: 1
@@ -237,7 +237,7 @@ STAGE PLANS:
File Output Operator
compressed: false
GlobalTableId: 1
- directory: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10002
+ directory: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10002
NumFilesPerFileSink: 1
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -248,12 +248,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
TotalFiles: 1
@@ -262,12 +262,15 @@ STAGE PLANS:
Alias Bucket Base File Name Mapping:
b {srcbucket20.txt=[srcbucket20.txt, srcbucket22.txt], srcbucket21.txt=[srcbucket21.txt, srcbucket23.txt]}
Alias Bucket File Name Mapping:
- b {file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket20.txt=[file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket20.txt, file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket22.txt], file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket21.txt=[file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket21.txt, file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket23.txt]}
+ b {file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket20.txt=[file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket20.txt, file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket22.txt], file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket21.txt=[file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket21.txt, file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket23.txt]}
+ Alias Bucket Output File Name Mapping:
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket20.txt 0
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket21.txt 1
Needs Tagging: false
Path -> Alias:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin [a]
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin [a]
Path -> Partition:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin
Partition
base file name: srcbucket_mapjoin
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -279,12 +282,12 @@ STAGE PLANS:
columns.types int:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin
name srcbucket_mapjoin
serialization.ddl struct srcbucket_mapjoin { i32 key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348782
+ transient_lastDdlTime 1270492668
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -296,12 +299,12 @@ STAGE PLANS:
columns.types int:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin
name srcbucket_mapjoin
serialization.ddl struct srcbucket_mapjoin { i32 key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348782
+ transient_lastDdlTime 1270492668
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: srcbucket_mapjoin
name: srcbucket_mapjoin
@@ -313,14 +316,14 @@ STAGE PLANS:
Move Operator
files:
hdfs directory: true
- source: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10002
- destination: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10000
+ source: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10002
+ destination: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10000
Stage: Stage-0
Move Operator
tables:
replace: true
- source: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10000
+ source: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10000
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -330,20 +333,20 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
- tmp directory: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10001
+ tmp directory: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10001
Stage: Stage-2
Map Reduce
Alias -> Map Operator Tree:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10002
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10002
Reduce Output Operator
sort order:
Map-reduce partition columns:
@@ -359,9 +362,9 @@ STAGE PLANS:
type: string
Needs Tagging: false
Path -> Alias:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10002 [file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10002]
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10002 [file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10002]
Path -> Partition:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10002
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10002
Partition
base file name: 10002
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -372,12 +375,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -388,12 +391,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
name: bucketmapjoin_tmp_result
@@ -402,7 +405,7 @@ STAGE PLANS:
File Output Operator
compressed: false
GlobalTableId: 0
- directory: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-24_320_7804903070266586536/10000
+ directory: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-37-51_124_5304744619994590803/10000
NumFilesPerFileSink: 1
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -413,12 +416,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
TotalFiles: 1
@@ -444,11 +447,11 @@ POSTHOOK: Output: default@bucketmapjoin_
PREHOOK: query: select count(1) from bucketmapjoin_tmp_result
PREHOOK: type: QUERY
PREHOOK: Input: default@bucketmapjoin_tmp_result
-PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-32_954_8069950489380829899/10000
+PREHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-00_833_4934747158815120399/10000
POSTHOOK: query: select count(1) from bucketmapjoin_tmp_result
POSTHOOK: type: QUERY
POSTHOOK: Input: default@bucketmapjoin_tmp_result
-POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-32_954_8069950489380829899/10000
+POSTHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-00_833_4934747158815120399/10000
464
PREHOOK: query: insert overwrite table bucketmapjoin_hash_result_1
select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result
@@ -479,11 +482,11 @@ POSTHOOK: Output: default@bucketmapjoin_
PREHOOK: query: select count(1) from bucketmapjoin_tmp_result
PREHOOK: type: QUERY
PREHOOK: Input: default@bucketmapjoin_tmp_result
-PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-49_054_4090200478765355576/10000
+PREHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-17_257_5555997609290633171/10000
POSTHOOK: query: select count(1) from bucketmapjoin_tmp_result
POSTHOOK: type: QUERY
POSTHOOK: Input: default@bucketmapjoin_tmp_result
-POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-49_054_4090200478765355576/10000
+POSTHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-17_257_5555997609290633171/10000
464
PREHOOK: query: insert overwrite table bucketmapjoin_hash_result_2
select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result
@@ -501,14 +504,14 @@ on a.key = b.key
PREHOOK: type: QUERY
PREHOOK: Input: default@bucketmapjoin_hash_result_2
PREHOOK: Input: default@bucketmapjoin_hash_result_1
-PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-56_624_3600396860306016630/10000
+PREHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-24_468_3503403929229761174/10000
POSTHOOK: query: select a.key-b.key, a.value1-b.value1, a.value2-b.value2
from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b
on a.key = b.key
POSTHOOK: type: QUERY
POSTHOOK: Input: default@bucketmapjoin_hash_result_2
POSTHOOK: Input: default@bucketmapjoin_hash_result_1
-POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-06-56_624_3600396860306016630/10000
+POSTHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-24_468_3503403929229761174/10000
0 0 0
PREHOOK: query: explain extended
insert overwrite table bucketmapjoin_tmp_result
@@ -584,7 +587,7 @@ STAGE PLANS:
File Output Operator
compressed: false
GlobalTableId: 1
- directory: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10002
+ directory: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10002
NumFilesPerFileSink: 1
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -595,12 +598,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
TotalFiles: 1
@@ -655,7 +658,7 @@ STAGE PLANS:
File Output Operator
compressed: false
GlobalTableId: 1
- directory: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10002
+ directory: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10002
NumFilesPerFileSink: 1
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -666,12 +669,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
TotalFiles: 1
@@ -680,12 +683,17 @@ STAGE PLANS:
Alias Bucket Base File Name Mapping:
a {srcbucket20.txt=[srcbucket20.txt], srcbucket21.txt=[srcbucket21.txt], srcbucket22.txt=[srcbucket20.txt], srcbucket23.txt=[srcbucket21.txt]}
Alias Bucket File Name Mapping:
- a {file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket20.txt=[file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket20.txt], file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket21.txt=[file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket21.txt], file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket22.txt=[file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket20.txt], file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket23.txt=[file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/
warehouse/srcbucket_mapjoin/srcbucket21.txt]}
+ a {file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket20.txt=[file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket20.txt], file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket21.txt=[file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket21.txt], file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket22.txt=[file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket20.txt], file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket23.txt=[file:/dat
a/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin/srcbucket21.txt]}
+ Alias Bucket Output File Name Mapping:
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket20.txt 0
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket21.txt 1
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket22.txt 2
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08/srcbucket23.txt 3
Needs Tagging: false
Path -> Alias:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08 [b]
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08 [b]
Path -> Partition:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part/ds=2008-04-08
Partition
base file name: ds=2008-04-08
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -699,13 +707,13 @@ STAGE PLANS:
columns.types int:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part
name srcbucket_mapjoin_part
partition_columns ds
serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348782
+ transient_lastDdlTime 1270492669
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -717,13 +725,13 @@ STAGE PLANS:
columns.types int:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/srcbucket_mapjoin_part
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/srcbucket_mapjoin_part
name srcbucket_mapjoin_part
partition_columns ds
serialization.ddl struct srcbucket_mapjoin_part { i32 key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348782
+ transient_lastDdlTime 1270492669
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: srcbucket_mapjoin_part
name: srcbucket_mapjoin_part
@@ -735,14 +743,14 @@ STAGE PLANS:
Move Operator
files:
hdfs directory: true
- source: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10002
- destination: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10000
+ source: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10002
+ destination: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10000
Stage: Stage-0
Move Operator
tables:
replace: true
- source: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10000
+ source: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10000
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -752,20 +760,20 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
- tmp directory: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10001
+ tmp directory: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10001
Stage: Stage-2
Map Reduce
Alias -> Map Operator Tree:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10002
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10002
Reduce Output Operator
sort order:
Map-reduce partition columns:
@@ -781,9 +789,9 @@ STAGE PLANS:
type: string
Needs Tagging: false
Path -> Alias:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10002 [file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10002]
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10002 [file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10002]
Path -> Partition:
- file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10002
+ file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10002
Partition
base file name: 10002
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -794,12 +802,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -810,12 +818,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
name: bucketmapjoin_tmp_result
@@ -824,7 +832,7 @@ STAGE PLANS:
File Output Operator
compressed: false
GlobalTableId: 0
- directory: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-01_012_3767509085170715505/10000
+ directory: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-28_297_6620917679758077647/10000
NumFilesPerFileSink: 1
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -835,12 +843,12 @@ STAGE PLANS:
columns.types string:string:string
file.inputformat org.apache.hadoop.mapred.TextInputFormat
file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- location file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
+ location file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/test/data/warehouse/bucketmapjoin_tmp_result
name bucketmapjoin_tmp_result
serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- transient_lastDdlTime 1268348784
+ transient_lastDdlTime 1270492671
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: bucketmapjoin_tmp_result
TotalFiles: 1
@@ -866,11 +874,11 @@ POSTHOOK: Output: default@bucketmapjoin_
PREHOOK: query: select count(1) from bucketmapjoin_tmp_result
PREHOOK: type: QUERY
PREHOOK: Input: default@bucketmapjoin_tmp_result
-PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-10_899_5943241423234312650/10000
+PREHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-37_665_1360583148724299888/10000
POSTHOOK: query: select count(1) from bucketmapjoin_tmp_result
POSTHOOK: type: QUERY
POSTHOOK: Input: default@bucketmapjoin_tmp_result
-POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-10_899_5943241423234312650/10000
+POSTHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-37_665_1360583148724299888/10000
464
PREHOOK: query: insert overwrite table bucketmapjoin_hash_result_1
select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result
@@ -901,11 +909,11 @@ POSTHOOK: Output: default@bucketmapjoin_
PREHOOK: query: select count(1) from bucketmapjoin_tmp_result
PREHOOK: type: QUERY
PREHOOK: Input: default@bucketmapjoin_tmp_result
-PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-28_066_1208658957261253807/10000
+PREHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-54_047_7919202086234490141/10000
POSTHOOK: query: select count(1) from bucketmapjoin_tmp_result
POSTHOOK: type: QUERY
POSTHOOK: Input: default@bucketmapjoin_tmp_result
-POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-28_066_1208658957261253807/10000
+POSTHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-38-54_047_7919202086234490141/10000
464
PREHOOK: query: insert overwrite table bucketmapjoin_hash_result_2
select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result
@@ -923,14 +931,14 @@ on a.key = b.key
PREHOOK: type: QUERY
PREHOOK: Input: default@bucketmapjoin_hash_result_2
PREHOOK: Input: default@bucketmapjoin_hash_result_1
-PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-36_714_9051036103640437662/10000
+PREHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-39-01_488_7836991632267919786/10000
POSTHOOK: query: select a.key-b.key, a.value1-b.value1, a.value2-b.value2
from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b
on a.key = b.key
POSTHOOK: type: QUERY
POSTHOOK: Input: default@bucketmapjoin_hash_result_2
POSTHOOK: Input: default@bucketmapjoin_hash_result_1
-POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-460/build/ql/scratchdir/hive_2010-03-11_15-07-36_714_9051036103640437662/10000
+POSTHOOK: Output: file:/data/users/njain/deploy/deploy1/trunk/VENDOR.hive/trunk/build/ql/scratchdir/hive_2010-04-05_11-39-01_488_7836991632267919786/10000
0 0 0
PREHOOK: query: drop table bucketmapjoin_hash_result_2
PREHOOK: type: DROPTABLE