PageRenderTime 65ms CodeModel.GetById 9ms app.highlight 51ms RepoModel.GetById 1ms app.codeStats 0ms

/tags/release-0.0.0-rc0/hive/external/ql/src/java/org/apache/hadoop/hive/ql/exec/SkewJoinHandler.java

#
Java | 357 lines | 260 code | 42 blank | 55 comment | 43 complexity | 9a853633667c944f2afa961c592b56d5 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.exec;
 20
 21import java.io.FileNotFoundException;
 22import java.io.IOException;
 23import java.io.Serializable;
 24import java.util.ArrayList;
 25import java.util.Arrays;
 26import java.util.HashMap;
 27import java.util.List;
 28import java.util.Map;
 29
 30import org.apache.commons.logging.Log;
 31import org.apache.commons.logging.LogFactory;
 32import org.apache.hadoop.conf.Configuration;
 33import org.apache.hadoop.fs.FileSystem;
 34import org.apache.hadoop.fs.Path;
 35import org.apache.hadoop.hive.ql.exec.persistence.RowContainer;
 36import org.apache.hadoop.hive.ql.metadata.HiveException;
 37import org.apache.hadoop.hive.ql.plan.JoinDesc;
 38import org.apache.hadoop.hive.ql.plan.TableDesc;
 39import org.apache.hadoop.hive.serde2.SerDe;
 40import org.apache.hadoop.hive.serde2.SerDeException;
 41import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 42import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 43import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 44import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 45import org.apache.hadoop.io.LongWritable;
 46import org.apache.hadoop.util.ReflectionUtils;
 47
 48/**
 49 * At runtime in Join, we output big keys in one table into one corresponding
 50 * directories, and all same keys in other tables into different dirs(one for
 51 * each table). The directories will look like:
 52 * <ul>
 53 * <li>
 54 * dir-T1-bigkeys(containing big keys in T1), dir-T2-keys(containing keys which
 55 * is big in T1),dir-T3-keys(containing keys which is big in T1), ...
 56 * <li>
 57 * dir-T1-keys(containing keys which is big in T2), dir-T2-bigkeys(containing
 58 * big keys in T2),dir-T3-keys(containing keys which is big in T2), ...
 59 * <li>
 60 * dir-T1-keys(containing keys which is big in T3), dir-T2-keys(containing big
 61 * keys in T3),dir-T3-bigkeys(containing keys which is big in T3), ... .....
 62 * </ul>
 63 *
 64 * <p>
 65 * For each skew key, we first write all values to a local tmp file. At the time
 66 * of ending the current group, the local tmp file will be uploaded to hdfs.
 67 * Right now, we use one file per skew key.
 68 *
 69 * <p>
 70 * For more info, please see https://issues.apache.org/jira/browse/HIVE-964.
 71 *
 72 */
 73public class SkewJoinHandler {
 74
 75  protected static final Log LOG = LogFactory.getLog(SkewJoinHandler.class
 76      .getName());
 77
 78  public int currBigKeyTag = -1;
 79
 80  private int rowNumber = 0;
 81  private int currTag = -1;
 82
 83  private int skewKeyDefinition = -1;
 84  private Map<Byte, StructObjectInspector> skewKeysTableObjectInspector = null;
 85  private Map<Byte, SerDe> tblSerializers = null;
 86  private Map<Byte, TableDesc> tblDesc = null;
 87
 88  private Map<Byte, Boolean> bigKeysExistingMap = null;
 89
 90  private LongWritable skewjoinFollowupJobs;
 91
 92  private final boolean noOuterJoin;
 93  Configuration hconf = null;
 94  List<Object> dummyKey = null;
 95  String taskId;
 96
 97  private final CommonJoinOperator<? extends Serializable> joinOp;
 98  private final int numAliases;
 99  private final JoinDesc conf;
100
101  public SkewJoinHandler(CommonJoinOperator<? extends Serializable> joinOp) {
102    this.joinOp = joinOp;
103    numAliases = joinOp.numAliases;
104    conf = joinOp.getConf();
105    noOuterJoin = joinOp.noOuterJoin;
106  }
107
108  public void initiliaze(Configuration hconf) {
109    this.hconf = hconf;
110    JoinDesc desc = joinOp.getConf();
111    skewKeyDefinition = desc.getSkewKeyDefinition();
112    skewKeysTableObjectInspector = new HashMap<Byte, StructObjectInspector>(
113        numAliases);
114    tblDesc = desc.getSkewKeysValuesTables();
115    tblSerializers = new HashMap<Byte, SerDe>(numAliases);
116    bigKeysExistingMap = new HashMap<Byte, Boolean>(numAliases);
117    taskId = Utilities.getTaskId(hconf);
118
119    for (int i = 0; i < numAliases; i++) {
120      Byte alias = conf.getTagOrder()[i];
121      List<ObjectInspector> skewTableKeyInspectors = new ArrayList<ObjectInspector>();
122      StructObjectInspector soi = (StructObjectInspector) joinOp.inputObjInspectors[alias];
123      StructField sf = soi.getStructFieldRef(Utilities.ReduceField.KEY
124          .toString());
125      List<? extends StructField> keyFields = ((StructObjectInspector) sf
126          .getFieldObjectInspector()).getAllStructFieldRefs();
127      int keyFieldSize = keyFields.size();
128      for (int k = 0; k < keyFieldSize; k++) {
129        skewTableKeyInspectors.add(keyFields.get(k).getFieldObjectInspector());
130      }
131      TableDesc joinKeyDesc = desc.getKeyTableDesc();
132      List<String> keyColNames = Utilities.getColumnNames(joinKeyDesc
133          .getProperties());
134      StructObjectInspector structTblKeyInpector = ObjectInspectorFactory
135          .getStandardStructObjectInspector(keyColNames, skewTableKeyInspectors);
136
137      try {
138        SerDe serializer = (SerDe) ReflectionUtils.newInstance(tblDesc.get(
139            alias).getDeserializerClass(), null);
140        serializer.initialize(null, tblDesc.get(alias).getProperties());
141        tblSerializers.put((byte) i, serializer);
142      } catch (SerDeException e) {
143        LOG.error("Skewjoin will be disabled due to " + e.getMessage(), e);
144        joinOp.handleSkewJoin = false;
145        break;
146      }
147
148      TableDesc valTblDesc = JoinUtil.getSpillTableDesc(alias,joinOp.spillTableDesc,conf,noOuterJoin);
149      List<String> valColNames = new ArrayList<String>();
150      if (valTblDesc != null) {
151        valColNames = Utilities.getColumnNames(valTblDesc.getProperties());
152      }
153      StructObjectInspector structTblValInpector = ObjectInspectorFactory
154          .getStandardStructObjectInspector(valColNames,
155          joinOp.joinValuesStandardObjectInspectors.get((byte) i));
156
157      StructObjectInspector structTblInpector = ObjectInspectorFactory
158          .getUnionStructObjectInspector(Arrays
159          .asList(new StructObjectInspector[] {structTblValInpector, structTblKeyInpector}));
160      skewKeysTableObjectInspector.put((byte) i, structTblInpector);
161    }
162
163    // reset rowcontainer's serde, objectinspector, and tableDesc.
164    for (int i = 0; i < numAliases; i++) {
165      Byte alias = conf.getTagOrder()[i];
166      RowContainer<ArrayList<Object>> rc = (RowContainer)joinOp.storage.get(Byte
167          .valueOf((byte) i));
168      if (rc != null) {
169        rc.setSerDe(tblSerializers.get((byte) i), skewKeysTableObjectInspector
170            .get((byte) i));
171        rc.setTableDesc(tblDesc.get(alias));
172      }
173    }
174  }
175
176  void endGroup() throws IOException, HiveException {
177    if (skewKeyInCurrentGroup) {
178
179      String specPath = conf.getBigKeysDirMap().get((byte) currBigKeyTag);
180      RowContainer<ArrayList<Object>> bigKey = (RowContainer)joinOp.storage.get(Byte
181          .valueOf((byte) currBigKeyTag));
182      Path outputPath = getOperatorOutputPath(specPath);
183      FileSystem destFs = outputPath.getFileSystem(hconf);
184      bigKey.copyToDFSDirecory(destFs, outputPath);
185
186      for (int i = 0; i < numAliases; i++) {
187        if (((byte) i) == currBigKeyTag) {
188          continue;
189        }
190        RowContainer<ArrayList<Object>> values = (RowContainer)joinOp.storage.get(Byte
191            .valueOf((byte) i));
192        if (values != null) {
193          specPath = conf.getSmallKeysDirMap().get((byte) currBigKeyTag).get(
194              (byte) i);
195          values.copyToDFSDirecory(destFs, getOperatorOutputPath(specPath));
196        }
197      }
198    }
199    skewKeyInCurrentGroup = false;
200  }
201
202  boolean skewKeyInCurrentGroup = false;
203
204  public void handleSkew(int tag) throws HiveException {
205
206    if (joinOp.newGroupStarted || tag != currTag) {
207      rowNumber = 0;
208      currTag = tag;
209    }
210
211    if (joinOp.newGroupStarted) {
212      currBigKeyTag = -1;
213      joinOp.newGroupStarted = false;
214      dummyKey = (List<Object>) joinOp.getGroupKeyObject();
215      skewKeyInCurrentGroup = false;
216
217      for (int i = 0; i < numAliases; i++) {
218        RowContainer<ArrayList<Object>> rc = (RowContainer)joinOp.storage.get(Byte
219            .valueOf((byte) i));
220        if (rc != null) {
221          rc.setKeyObject(dummyKey);
222        }
223      }
224    }
225
226    rowNumber++;
227    if (currBigKeyTag == -1 && (tag < numAliases - 1)
228        && rowNumber >= skewKeyDefinition) {
229      // the first time we see a big key. If this key is not in the last
230      // table (the last table can always be streamed), we define that we get
231      // a skew key now.
232      currBigKeyTag = tag;
233      updateSkewJoinJobCounter(tag);
234      // right now we assume that the group by is an ArrayList object. It may
235      // change in future.
236      if (!(dummyKey instanceof List)) {
237        throw new RuntimeException("Bug in handle skew key in a seperate job.");
238      }
239
240      skewKeyInCurrentGroup = true;
241      bigKeysExistingMap.put(Byte.valueOf((byte) currBigKeyTag), Boolean.TRUE);
242    }
243  }
244
245  public void close(boolean abort) throws HiveException {
246    if (!abort) {
247      try {
248        endGroup();
249        commit();
250      } catch (IOException e) {
251        throw new HiveException(e);
252      }
253    } else {
254      for (int bigKeyTbl = 0; bigKeyTbl < numAliases; bigKeyTbl++) {
255
256        // if we did not see a skew key in this table, continue to next
257        // table
258        if (!bigKeysExistingMap.get((byte) bigKeyTbl)) {
259          continue;
260        }
261
262        try {
263          String specPath = conf.getBigKeysDirMap().get((byte) bigKeyTbl);
264          Path bigKeyPath = getOperatorOutputPath(specPath);
265          FileSystem fs = bigKeyPath.getFileSystem(hconf);
266          delete(bigKeyPath, fs);
267          for (int smallKeyTbl = 0; smallKeyTbl < numAliases; smallKeyTbl++) {
268            if (((byte) smallKeyTbl) == bigKeyTbl) {
269              continue;
270            }
271            specPath = conf.getSmallKeysDirMap().get((byte) bigKeyTbl).get(
272                (byte) smallKeyTbl);
273            delete(getOperatorOutputPath(specPath), fs);
274          }
275        } catch (IOException e) {
276          throw new HiveException(e);
277        }
278      }
279    }
280  }
281
282  private void delete(Path operatorOutputPath, FileSystem fs) {
283    try {
284      fs.delete(operatorOutputPath, true);
285    } catch (IOException e) {
286      LOG.error(e);
287    }
288  }
289
290  private void commit() throws IOException {
291    for (int bigKeyTbl = 0; bigKeyTbl < numAliases; bigKeyTbl++) {
292
293      // if we did not see a skew key in this table, continue to next table
294      // we are trying to avoid an extra call of FileSystem.exists()
295      Boolean existing = bigKeysExistingMap.get(Byte.valueOf((byte) bigKeyTbl));
296      if (existing == null || !existing) {
297        continue;
298      }
299
300      String specPath = conf.getBigKeysDirMap().get(
301          Byte.valueOf((byte) bigKeyTbl));
302      commitOutputPathToFinalPath(specPath, false);
303      for (int smallKeyTbl = 0; smallKeyTbl < numAliases; smallKeyTbl++) {
304        if (smallKeyTbl == bigKeyTbl) {
305          continue;
306        }
307        specPath = conf.getSmallKeysDirMap()
308            .get(Byte.valueOf((byte) bigKeyTbl)).get(
309            Byte.valueOf((byte) smallKeyTbl));
310        // the file may not exist, and we just ignore this
311        commitOutputPathToFinalPath(specPath, true);
312      }
313    }
314  }
315
316  private void commitOutputPathToFinalPath(String specPath,
317      boolean ignoreNonExisting) throws IOException {
318    Path outPath = getOperatorOutputPath(specPath);
319    Path finalPath = getOperatorFinalPath(specPath);
320    FileSystem fs = outPath.getFileSystem(hconf);
321    // for local file system in Hadoop-0.17.2.1, it will throw IOException when
322    // file not existing.
323    try {
324      if (!fs.rename(outPath, finalPath)) {
325        throw new IOException("Unable to rename output to: " + finalPath);
326      }
327    } catch (FileNotFoundException e) {
328      if (!ignoreNonExisting) {
329        throw e;
330      }
331    } catch (IOException e) {
332      if (!fs.exists(outPath) && ignoreNonExisting) {
333        return;
334      }
335      throw e;
336    }
337  }
338
339  private Path getOperatorOutputPath(String specPath) throws IOException {
340    Path tmpPath = Utilities.toTempPath(specPath);
341    return new Path(tmpPath, Utilities.toTempPath(taskId));
342  }
343
344  private Path getOperatorFinalPath(String specPath) throws IOException {
345    Path tmpPath = Utilities.toTempPath(specPath);
346    return new Path(tmpPath, taskId);
347  }
348
349  public void setSkewJoinJobCounter(LongWritable skewjoinFollowupJobs) {
350    this.skewjoinFollowupJobs = skewjoinFollowupJobs;
351  }
352
353  public void updateSkewJoinJobCounter(int tag) {
354    this.skewjoinFollowupJobs.set(this.skewjoinFollowupJobs.get() + 1);
355  }
356
357}