PageRenderTime 46ms CodeModel.GetById 10ms app.highlight 31ms RepoModel.GetById 1ms app.codeStats 0ms

/tags/release-0.0.0-rc0/hive/external/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java

#
Java | 401 lines | 284 code | 52 blank | 65 comment | 29 complexity | 26f85ed5000018df859468436fbec43d MD5 | raw file
  1/**
  2 * Licensed to the Apache Software Foundation (ASF) under one
  3 * or more contributor license agreements.  See the NOTICE file
  4 * distributed with this work for additional information
  5 * regarding copyright ownership.  The ASF licenses this file
  6 * to you under the Apache License, Version 2.0 (the
  7 * "License"); you may not use this file except in compliance
  8 * with the License.  You may obtain a copy of the License at
  9 *
 10 *     http://www.apache.org/licenses/LICENSE-2.0
 11 *
 12 * Unless required by applicable law or agreed to in writing, software
 13 * distributed under the License is distributed on an "AS IS" BASIS,
 14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 15 * See the License for the specific language governing permissions and
 16 * limitations under the License.
 17 */
 18
 19package org.apache.hadoop.hive.ql.optimizer.physical;
 20
 21import java.io.ByteArrayInputStream;
 22import java.io.File;
 23import java.io.Serializable;
 24import java.io.UnsupportedEncodingException;
 25import java.util.ArrayList;
 26import java.util.HashMap;
 27import java.util.LinkedHashMap;
 28import java.util.List;
 29import java.util.Map;
 30
 31import org.apache.hadoop.fs.Path;
 32import org.apache.hadoop.hive.conf.HiveConf;
 33import org.apache.hadoop.hive.ql.exec.ConditionalTask;
 34import org.apache.hadoop.hive.ql.exec.JoinOperator;
 35import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 36import org.apache.hadoop.hive.ql.exec.Operator;
 37import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 38import org.apache.hadoop.hive.ql.exec.RowSchema;
 39import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 40import org.apache.hadoop.hive.ql.exec.Task;
 41import org.apache.hadoop.hive.ql.exec.TaskFactory;
 42import org.apache.hadoop.hive.ql.exec.Utilities;
 43import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 44import org.apache.hadoop.hive.ql.parse.ParseContext;
 45import org.apache.hadoop.hive.ql.parse.SemanticException;
 46import org.apache.hadoop.hive.ql.plan.ConditionalResolverSkewJoin;
 47import org.apache.hadoop.hive.ql.plan.ConditionalWork;
 48import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 49import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 50import org.apache.hadoop.hive.ql.plan.FetchWork;
 51import org.apache.hadoop.hive.ql.plan.JoinDesc;
 52import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 53import org.apache.hadoop.hive.ql.plan.MapredLocalWork;
 54import org.apache.hadoop.hive.ql.plan.MapredWork;
 55import org.apache.hadoop.hive.ql.plan.PartitionDesc;
 56import org.apache.hadoop.hive.ql.plan.PlanUtils;
 57import org.apache.hadoop.hive.ql.plan.TableDesc;
 58import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 59import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 60import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 61import org.apache.hadoop.hive.ql.metadata.HiveException;
 62
 63/**
 64 * GenMRSkewJoinProcessor.
 65 *
 66 */
 67public final class GenMRSkewJoinProcessor {
 68
 69  private GenMRSkewJoinProcessor() {
 70    // prevent instantiation
 71  }
 72
 73  /**
 74   * Create tasks for processing skew joins. The idea is (HIVE-964) to use
 75   * separated jobs and map-joins to handle skew joins.
 76   * <p>
 77   * <ul>
 78   * <li>
 79   * Number of mr jobs to handle skew keys is the number of table minus 1 (we
 80   * can stream the last table, so big keys in the last table will not be a
 81   * problem).
 82   * <li>
 83   * At runtime in Join, we output big keys in one table into one corresponding
 84   * directories, and all same keys in other tables into different dirs(one for
 85   * each table). The directories will look like:
 86   * <ul>
 87   * <li>
 88   * dir-T1-bigkeys(containing big keys in T1), dir-T2-keys(containing keys
 89   * which is big in T1),dir-T3-keys(containing keys which is big in T1), ...
 90   * <li>
 91   * dir-T1-keys(containing keys which is big in T2), dir-T2-bigkeys(containing
 92   * big keys in T2),dir-T3-keys(containing keys which is big in T2), ...
 93   * <li>
 94   * dir-T1-keys(containing keys which is big in T3), dir-T2-keys(containing big
 95   * keys in T3),dir-T3-bigkeys(containing keys which is big in T3), ... .....
 96   * </ul>
 97   * </ul>
 98   * For each table, we launch one mapjoin job, taking the directory containing
 99   * big keys in this table and corresponding dirs in other tables as input.
100   * (Actally one job for one row in the above.)
101   *
102   * <p>
103   * For more discussions, please check
104   * https://issues.apache.org/jira/browse/HIVE-964.
105   *
106   */
107  public static void processSkewJoin(JoinOperator joinOp,
108      Task<? extends Serializable> currTask, ParseContext parseCtx)
109      throws SemanticException {
110
111    // We are trying to adding map joins to handle skew keys, and map join right
112    // now does not work with outer joins
113    if (!GenMRSkewJoinProcessor.skewJoinEnabled(parseCtx.getConf(), joinOp)) {
114      return;
115    }
116
117    String baseTmpDir = parseCtx.getContext().getMRTmpFileURI();
118
119    JoinDesc joinDescriptor = joinOp.getConf();
120    Map<Byte, List<ExprNodeDesc>> joinValues = joinDescriptor.getExprs();
121    int numAliases = joinValues.size();
122
123    Map<Byte, String> bigKeysDirMap = new HashMap<Byte, String>();
124    Map<Byte, Map<Byte, String>> smallKeysDirMap = new HashMap<Byte, Map<Byte, String>>();
125    Map<Byte, String> skewJoinJobResultsDir = new HashMap<Byte, String>();
126    Byte[] tags = joinDescriptor.getTagOrder();
127    for (int i = 0; i < numAliases; i++) {
128      Byte alias = tags[i];
129      String bigKeysDir = getBigKeysDir(baseTmpDir, alias);
130      bigKeysDirMap.put(alias, bigKeysDir);
131      Map<Byte, String> smallKeysMap = new HashMap<Byte, String>();
132      smallKeysDirMap.put(alias, smallKeysMap);
133      for (Byte src2 : tags) {
134        if (!src2.equals(alias)) {
135          smallKeysMap.put(src2, getSmallKeysDir(baseTmpDir, alias, src2));
136        }
137      }
138      skewJoinJobResultsDir.put(alias, getBigKeysSkewJoinResultDir(baseTmpDir,
139          alias));
140    }
141
142    joinDescriptor.setHandleSkewJoin(true);
143    joinDescriptor.setBigKeysDirMap(bigKeysDirMap);
144    joinDescriptor.setSmallKeysDirMap(smallKeysDirMap);
145    joinDescriptor.setSkewKeyDefinition(HiveConf.getIntVar(parseCtx.getConf(),
146        HiveConf.ConfVars.HIVESKEWJOINKEY));
147
148    HashMap<String, Task<? extends Serializable>> bigKeysDirToTaskMap =
149      new HashMap<String, Task<? extends Serializable>>();
150    List<Serializable> listWorks = new ArrayList<Serializable>();
151    List<Task<? extends Serializable>> listTasks = new ArrayList<Task<? extends Serializable>>();
152    MapredWork currPlan = (MapredWork) currTask.getWork();
153
154    TableDesc keyTblDesc = (TableDesc) currPlan.getKeyDesc().clone();
155    List<String> joinKeys = Utilities
156        .getColumnNames(keyTblDesc.getProperties());
157    List<String> joinKeyTypes = Utilities.getColumnTypes(keyTblDesc
158        .getProperties());
159
160    Map<Byte, TableDesc> tableDescList = new HashMap<Byte, TableDesc>();
161    Map<Byte, List<ExprNodeDesc>> newJoinValues = new HashMap<Byte, List<ExprNodeDesc>>();
162    Map<Byte, List<ExprNodeDesc>> newJoinKeys = new HashMap<Byte, List<ExprNodeDesc>>();
163    // used for create mapJoinDesc, should be in order
164    List<TableDesc> newJoinValueTblDesc = new ArrayList<TableDesc>();
165
166    for (Byte tag : tags) {
167      newJoinValueTblDesc.add(null);
168    }
169
170    for (int i = 0; i < numAliases; i++) {
171      Byte alias = tags[i];
172      List<ExprNodeDesc> valueCols = joinValues.get(alias);
173      String colNames = "";
174      String colTypes = "";
175      int columnSize = valueCols.size();
176      List<ExprNodeDesc> newValueExpr = new ArrayList<ExprNodeDesc>();
177      List<ExprNodeDesc> newKeyExpr = new ArrayList<ExprNodeDesc>();
178
179      boolean first = true;
180      for (int k = 0; k < columnSize; k++) {
181        TypeInfo type = valueCols.get(k).getTypeInfo();
182        String newColName = i + "_VALUE_" + k; // any name, it does not matter.
183        newValueExpr
184            .add(new ExprNodeColumnDesc(type, newColName, "" + i, false));
185        if (!first) {
186          colNames = colNames + ",";
187          colTypes = colTypes + ",";
188        }
189        first = false;
190        colNames = colNames + newColName;
191        colTypes = colTypes + valueCols.get(k).getTypeString();
192      }
193
194      // we are putting join keys at last part of the spilled table
195      for (int k = 0; k < joinKeys.size(); k++) {
196        if (!first) {
197          colNames = colNames + ",";
198          colTypes = colTypes + ",";
199        }
200        first = false;
201        colNames = colNames + joinKeys.get(k);
202        colTypes = colTypes + joinKeyTypes.get(k);
203        newKeyExpr.add(new ExprNodeColumnDesc(TypeInfoFactory
204            .getPrimitiveTypeInfo(joinKeyTypes.get(k)), joinKeys.get(k),
205            "" + i, false));
206      }
207
208      newJoinValues.put(alias, newValueExpr);
209      newJoinKeys.put(alias, newKeyExpr);
210      tableDescList.put(alias, Utilities.getTableDesc(colNames, colTypes));
211
212      // construct value table Desc
213      String valueColNames = "";
214      String valueColTypes = "";
215      first = true;
216      for (int k = 0; k < columnSize; k++) {
217        String newColName = i + "_VALUE_" + k; // any name, it does not matter.
218        if (!first) {
219          valueColNames = valueColNames + ",";
220          valueColTypes = valueColTypes + ",";
221        }
222        valueColNames = valueColNames + newColName;
223        valueColTypes = valueColTypes + valueCols.get(k).getTypeString();
224        first = false;
225      }
226      newJoinValueTblDesc.set(Byte.valueOf((byte) i), Utilities.getTableDesc(
227          valueColNames, valueColTypes));
228    }
229
230    joinDescriptor.setSkewKeysValuesTables(tableDescList);
231    joinDescriptor.setKeyTableDesc(keyTblDesc);
232
233    for (int i = 0; i < numAliases - 1; i++) {
234      Byte src = tags[i];
235      MapredWork newPlan = PlanUtils.getMapRedWork();
236
237      // This code has been only added for testing
238      boolean mapperCannotSpanPartns =
239        parseCtx.getConf().getBoolVar(
240          HiveConf.ConfVars.HIVE_MAPPER_CANNOT_SPAN_MULTIPLE_PARTITIONS);
241      newPlan.setMapperCannotSpanPartns(mapperCannotSpanPartns);
242
243      MapredWork clonePlan = null;
244      try {
245        String xmlPlan = currPlan.toXML();
246        StringBuilder sb = new StringBuilder(xmlPlan);
247        ByteArrayInputStream bis;
248        bis = new ByteArrayInputStream(sb.toString().getBytes("UTF-8"));
249        clonePlan = Utilities.deserializeMapRedWork(bis, parseCtx.getConf());
250      } catch (UnsupportedEncodingException e) {
251        throw new SemanticException(e);
252      }
253
254      Operator<? extends Serializable>[] parentOps = new TableScanOperator[tags.length];
255      for (int k = 0; k < tags.length; k++) {
256        Operator<? extends Serializable> ts = OperatorFactory.get(
257            TableScanDesc.class, (RowSchema) null);
258        ((TableScanOperator)ts).setTableDesc(tableDescList.get((byte)k));
259        parentOps[k] = ts;
260      }
261      Operator<? extends Serializable> tblScan_op = parentOps[i];
262
263      ArrayList<String> aliases = new ArrayList<String>();
264      String alias = src.toString();
265      aliases.add(alias);
266      String bigKeyDirPath = bigKeysDirMap.get(src);
267      newPlan.getPathToAliases().put(bigKeyDirPath, aliases);
268
269
270
271
272      newPlan.getAliasToWork().put(alias, tblScan_op);
273      PartitionDesc part = new PartitionDesc(tableDescList.get(src), null);
274
275
276      newPlan.getPathToPartitionInfo().put(bigKeyDirPath, part);
277      newPlan.getAliasToPartnInfo().put(alias, part);
278
279      Operator<? extends Serializable> reducer = clonePlan.getReducer();
280      assert reducer instanceof JoinOperator;
281      JoinOperator cloneJoinOp = (JoinOperator) reducer;
282
283      MapJoinDesc mapJoinDescriptor = new MapJoinDesc(newJoinKeys, keyTblDesc,
284          newJoinValues, newJoinValueTblDesc, newJoinValueTblDesc,joinDescriptor
285          .getOutputColumnNames(), i, joinDescriptor.getConds(),
286          joinDescriptor.getFilters(), joinDescriptor.getNoOuterJoin());
287      mapJoinDescriptor.setTagOrder(tags);
288      mapJoinDescriptor.setHandleSkewJoin(false);
289
290      MapredLocalWork localPlan = new MapredLocalWork(
291          new LinkedHashMap<String, Operator<? extends Serializable>>(),
292          new LinkedHashMap<String, FetchWork>());
293      Map<Byte, String> smallTblDirs = smallKeysDirMap.get(src);
294
295      for (int j = 0; j < numAliases; j++) {
296        if (j == i) {
297          continue;
298        }
299        Byte small_alias = tags[j];
300        Operator<? extends Serializable> tblScan_op2 = parentOps[j];
301        localPlan.getAliasToWork().put(small_alias.toString(), tblScan_op2);
302        Path tblDir = new Path(smallTblDirs.get(small_alias));
303        localPlan.getAliasToFetchWork().put(small_alias.toString(),
304            new FetchWork(tblDir.toString(), tableDescList.get(small_alias)));
305      }
306
307      newPlan.setMapLocalWork(localPlan);
308
309      // construct a map join and set it as the child operator of tblScan_op
310      MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory
311          .getAndMakeChild(mapJoinDescriptor, (RowSchema) null, parentOps);
312      // change the children of the original join operator to point to the map
313      // join operator
314      List<Operator<? extends Serializable>> childOps = cloneJoinOp
315          .getChildOperators();
316      for (Operator<? extends Serializable> childOp : childOps) {
317        childOp.replaceParent(cloneJoinOp, mapJoinOp);
318      }
319      mapJoinOp.setChildOperators(childOps);
320
321      HiveConf jc = new HiveConf(parseCtx.getConf(),
322          GenMRSkewJoinProcessor.class);
323
324      newPlan.setNumMapTasks(HiveConf
325          .getIntVar(jc, HiveConf.ConfVars.HIVESKEWJOINMAPJOINNUMMAPTASK));
326      newPlan
327          .setMinSplitSize(HiveConf.getLongVar(jc, HiveConf.ConfVars.HIVESKEWJOINMAPJOINMINSPLIT));
328      newPlan.setInputformat(HiveInputFormat.class.getName());
329      Task<? extends Serializable> skewJoinMapJoinTask = TaskFactory.get(
330          newPlan, jc);
331      bigKeysDirToTaskMap.put(bigKeyDirPath, skewJoinMapJoinTask);
332      listWorks.add(skewJoinMapJoinTask.getWork());
333      listTasks.add(skewJoinMapJoinTask);
334    }
335
336    ConditionalWork cndWork = new ConditionalWork(listWorks);
337    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork,
338        parseCtx.getConf());
339    cndTsk.setListTasks(listTasks);
340    cndTsk.setResolver(new ConditionalResolverSkewJoin());
341    cndTsk
342        .setResolverCtx(new ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx(
343        bigKeysDirToTaskMap));
344    List<Task<? extends Serializable>> oldChildTasks = currTask.getChildTasks();
345    currTask.setChildTasks(new ArrayList<Task<? extends Serializable>>());
346    currTask.addDependentTask(cndTsk);
347
348    if (oldChildTasks != null) {
349      for (Task<? extends Serializable> tsk : cndTsk.getListTasks()) {
350        for (Task<? extends Serializable> oldChild : oldChildTasks) {
351          tsk.addDependentTask(oldChild);
352        }
353      }
354    }
355    return;
356  }
357
358  public static boolean skewJoinEnabled(HiveConf conf, JoinOperator joinOp) {
359
360    if (conf != null && !conf.getBoolVar(HiveConf.ConfVars.HIVESKEWJOIN)) {
361      return false;
362    }
363
364    if (!joinOp.getConf().isNoOuterJoin()) {
365      return false;
366    }
367
368    byte pos = 0;
369    for (Byte tag : joinOp.getConf().getTagOrder()) {
370      if (tag != pos) {
371        return false;
372      }
373      pos++;
374    }
375
376    return true;
377  }
378
379  private static String skewJoinPrefix = "hive_skew_join";
380  private static String UNDERLINE = "_";
381  private static String BIGKEYS = "bigkeys";
382  private static String SMALLKEYS = "smallkeys";
383  private static String RESULTS = "results";
384
385  static String getBigKeysDir(String baseDir, Byte srcTbl) {
386    return baseDir + File.separator + skewJoinPrefix + UNDERLINE + BIGKEYS
387        + UNDERLINE + srcTbl;
388  }
389
390  static String getBigKeysSkewJoinResultDir(String baseDir, Byte srcTbl) {
391    return baseDir + File.separator + skewJoinPrefix + UNDERLINE + BIGKEYS
392        + UNDERLINE + RESULTS + UNDERLINE + srcTbl;
393  }
394
395  static String getSmallKeysDir(String baseDir, Byte srcTblBigTbl,
396      Byte srcTblSmallTbl) {
397    return baseDir + File.separator + skewJoinPrefix + UNDERLINE + SMALLKEYS
398        + UNDERLINE + srcTblBigTbl + UNDERLINE + srcTblSmallTbl;
399  }
400
401}