All Downloads are FREE. Search and download functionalities are using the official Maven repository.

org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor Maven / Gradle / Ivy

There is a newer version: 4.0.0
Show newest version
/*
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
 * distributed with this work for additional information
 * regarding copyright ownership.  The ASF licenses this file
 * to you under the Apache License, Version 2.0 (the
 * "License"); you may not use this file except in compliance
 * with the License.  You may obtain a copy of the License at
 *
 *     http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.hadoop.hive.ql.optimizer;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.Stack;

import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator;
import org.apache.hadoop.hive.ql.exec.ColumnInfo;
import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
import org.apache.hadoop.hive.ql.exec.GroupByOperator;
import org.apache.hadoop.hive.ql.exec.JoinOperator;
import org.apache.hadoop.hive.ql.exec.LateralViewJoinOperator;
import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
import org.apache.hadoop.hive.ql.exec.Operator;
import org.apache.hadoop.hive.ql.exec.OperatorFactory;
import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
import org.apache.hadoop.hive.ql.exec.RowSchema;
import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
import org.apache.hadoop.hive.ql.exec.ScriptOperator;
import org.apache.hadoop.hive.ql.exec.SelectOperator;
import org.apache.hadoop.hive.ql.exec.UnionOperator;
import org.apache.hadoop.hive.ql.exec.persistence.MapJoinKey;
import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
import org.apache.hadoop.hive.ql.lib.Node;
import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
import org.apache.hadoop.hive.ql.lib.SemanticRule;
import org.apache.hadoop.hive.ql.lib.RuleRegExp;
import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.EnabledOverride;
import org.apache.hadoop.hive.ql.parse.GenMapRedWalker;
import org.apache.hadoop.hive.ql.parse.ParseContext;
import org.apache.hadoop.hive.ql.parse.SemanticException;
import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
import org.apache.hadoop.hive.ql.plan.FetchWork;
import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
import org.apache.hadoop.hive.ql.plan.JoinDesc;
import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
import org.apache.hadoop.hive.ql.plan.MapredLocalWork;
import org.apache.hadoop.hive.ql.plan.MapredWork;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
import org.apache.hadoop.hive.ql.plan.PartitionDesc;
import org.apache.hadoop.hive.ql.plan.PlanUtils;
import org.apache.hadoop.hive.ql.plan.SMBJoinDesc;
import org.apache.hadoop.hive.ql.plan.SelectDesc;
import org.apache.hadoop.hive.ql.plan.TableDesc;
import org.apache.hadoop.hive.serde.serdeConstants;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
 * Implementation of one of the rule-based map join optimization. User passes hints to specify
 * map-joins and during this optimization, all user specified map joins are converted to MapJoins -
 * the reduce sink operator above the join are converted to map sink operators. In future, once
 * statistics are implemented, this transformation can also be done based on costs.
 */
public class MapJoinProcessor extends Transform {

  // mapjoin table descriptor contains a key descriptor which needs the field schema
  // (column type + column name). The column name is not really used anywhere, but it
  // needs to be passed. Use the string defined below for that.
  private static final String MAPJOINKEY_FIELDPREFIX = "mapjoinkey";
  private static final Logger LOG = LoggerFactory.getLogger(MapJoinProcessor.class.getName());

  public MapJoinProcessor() {
  }

  /**
   * Generate the MapRed Local Work for the given map-join operator
   *
   * @param newWork
   * @param mapJoinOp
   *          map-join operator for which local work needs to be generated.
   * @param bigTablePos
   * @throws SemanticException
   */
  private static void genMapJoinLocalWork(MapredWork newWork, MapJoinOperator mapJoinOp,
      int bigTablePos) throws SemanticException {
    // keep the small table alias to avoid concurrent modification exception
    ArrayList smallTableAliasList = new ArrayList();

    // create a new  MapredLocalWork
    MapredLocalWork newLocalWork = new MapredLocalWork(
        new LinkedHashMap>(),
        new LinkedHashMap());

    for (Map.Entry> entry :
      newWork.getMapWork().getAliasToWork().entrySet()) {
      String alias = entry.getKey();
      Operator op = entry.getValue();

      // if the table scan is for big table; then skip it
      // tracing down the operator tree from the table scan operator
      Operator parentOp = op;
      Operator childOp = op.getChildOperators().get(0);
      while ((childOp != null) && (!childOp.equals(mapJoinOp))) {
        parentOp = childOp;
        assert parentOp.getChildOperators().size() == 1;
        childOp = parentOp.getChildOperators().get(0);
      }
      if (childOp == null) {
        throw new SemanticException(
            "Cannot find join op by tracing down the table scan operator tree");
      }
      // skip the big table pos
      int i = childOp.getParentOperators().indexOf(parentOp);
      if (i == bigTablePos) {
        continue;
      }
      // set alias to work and put into smallTableAliasList
      newLocalWork.getAliasToWork().put(alias, op);
      smallTableAliasList.add(alias);
      // get input path and remove this alias from pathToAlias
      // because this file will be fetched by fetch operator
      Map> pathToAliases = newWork.getMapWork().getPathToAliases();

      // keep record all the input path for this alias
      HashSet pathSet = new HashSet<>();
      HashSet emptyPath = new HashSet<>();
      for (Map.Entry> entry2 : pathToAliases.entrySet()) {
        Path path = entry2.getKey();
        List list = entry2.getValue();
        if (list.contains(alias)) {
          // add to path set
          pathSet.add(path);
          //remove this alias from the alias list
          list.remove(alias);
          if(list.size() == 0) {
            emptyPath.add(path);
          }
        }
      }
      //remove the path, with which no alias associates
      for (Path path : emptyPath) {
        newWork.getMapWork().removePathToAlias(path);
      }

      // create fetch work
      FetchWork fetchWork = null;
      List partDir = new ArrayList();
      List partDesc = new ArrayList();

      for (Path tablePath : pathSet) {
        PartitionDesc partitionDesc = newWork.getMapWork().getPathToPartitionInfo().get(tablePath);
        // create fetchwork for non partitioned table
        if (partitionDesc.getPartSpec() == null || partitionDesc.getPartSpec().size() == 0) {
          fetchWork = new FetchWork(tablePath, partitionDesc.getTableDesc());
          break;
        }
        // if table is partitioned,add partDir and partitionDesc
        partDir.add(tablePath);
        partDesc.add(partitionDesc);
      }
      // create fetchwork for partitioned table
      if (fetchWork == null) {
        TableDesc table = newWork.getMapWork().getAliasToPartnInfo().get(alias).getTableDesc();
        fetchWork = new FetchWork(partDir, partDesc, table);
      }
      // set alias to fetch work
      newLocalWork.getAliasToFetchWork().put(alias, fetchWork);
    }
    // remove small table alias from aliasToWork;Avoid concurrent modification
    for (String alias : smallTableAliasList) {
      newWork.getMapWork().getAliasToWork().remove(alias);
    }

    // set up local work
    newWork.getMapWork().setMapRedLocalWork(newLocalWork);
    // remove reducer
    newWork.setReduceWork(null);
  }

  /**
   * Convert the join to a map-join and also generate any local work needed.
   *
   * @param newWork MapredWork in which the conversion is to happen
   * @param op
   *          The join operator that needs to be converted to map-join
   * @param mapJoinPos
   * @throws SemanticException
   */
  public static void genMapJoinOpAndLocalWork(HiveConf conf, MapredWork newWork,
    JoinOperator op, int mapJoinPos)
      throws SemanticException {
    // generate the map join operator; already checked the map join
    MapJoinOperator newMapJoinOp = new MapJoinProcessor().convertMapJoin(conf, op,
        newWork.getMapWork().isLeftInputJoin(), newWork.getMapWork().getBaseSrc(),
        newWork.getMapWork().getMapAliases(), mapJoinPos, true, false);
    genLocalWorkForMapJoin(newWork, newMapJoinOp, mapJoinPos);
  }

  public static void genLocalWorkForMapJoin(MapredWork newWork, MapJoinOperator newMapJoinOp,
      int mapJoinPos)
      throws SemanticException {
    try {
      // generate the local work for the big table alias
      MapJoinProcessor.genMapJoinLocalWork(newWork, newMapJoinOp, mapJoinPos);
      // clean up the mapred work
      newWork.getMapWork().setLeftInputJoin(false);
      newWork.getMapWork().setBaseSrc(null);
      newWork.getMapWork().setMapAliases(null);
    } catch (Exception e) {
      throw new SemanticException("Failed to generate new mapJoin operator " +
          "by exception : ", e);
    }
  }

  private static void checkParentOperatorType(Operator op)
      throws SemanticException {
    if (!op.opAllowedBeforeMapJoin()) {
      throw new SemanticException(ErrorMsg.OPERATOR_NOT_ALLOWED_WITH_MAPJOIN.getMsg());
    }
    if (op.getParentOperators() != null) {
      for (Operator parentOp : op.getParentOperators()) {
        checkParentOperatorType(parentOp);
      }
    }
  }

  private static void checkChildOperatorType(Operator op)
      throws SemanticException {
    if (!op.opAllowedAfterMapJoin()) {
      throw new SemanticException(ErrorMsg.OPERATOR_NOT_ALLOWED_WITH_MAPJOIN.getMsg());
    }
    for (Operator childOp : op.getChildOperators()) {
      checkChildOperatorType(childOp);
    }
  }

  private static void validateMapJoinTypes(Operator op)
      throws SemanticException {
    for (Operator parentOp : op.getParentOperators()) {
      checkParentOperatorType(parentOp);
    }

    for (Operator childOp : op.getChildOperators()) {
      checkChildOperatorType(childOp);
    }
  }

  /**
   * convert a regular join to a a map-side join.
   *
   * @param op
   *          join operator
   * @param mapJoinPos
   *          position of the source to be read as part of map-reduce framework. All other sources
   *          are cached in memory
   * @param noCheckOuterJoin
   * @param validateMapJoinTree
   */
  public MapJoinOperator convertMapJoin(HiveConf conf,
    JoinOperator op, boolean leftInputJoin, String[] baseSrc, List mapAliases,
    int mapJoinPos, boolean noCheckOuterJoin, boolean validateMapJoinTree) throws SemanticException {

    // outer join cannot be performed on a table which is being cached
    JoinDesc desc = op.getConf();
    JoinCondDesc[] condns = desc.getConds();

    if (!noCheckOuterJoin) {
      if (checkMapJoin(mapJoinPos, condns) < 0) {
        throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg());
      }
    }

    // Walk over all the sources (which are guaranteed to be reduce sink
    // operators).
    // The join outputs a concatenation of all the inputs.
    List> parentOps = op.getParentOperators();
    List> newParentOps =
      new ArrayList>();
    List> oldReduceSinkParentOps =
       new ArrayList>();

    // found a source which is not to be stored in memory
    if (leftInputJoin) {
      // assert mapJoinPos == 0;
      Operator parentOp = parentOps.get(0);
      assert parentOp.getParentOperators().size() == 1;
      Operator grandParentOp =
        parentOp.getParentOperators().get(0);
      oldReduceSinkParentOps.add(parentOp);
      newParentOps.add(grandParentOp);
    }

    byte pos = 0;
    // Remove parent reduce-sink operators
    for (String src : baseSrc) {
      if (src != null) {
        Operator parentOp = parentOps.get(pos);
        assert parentOp.getParentOperators().size() == 1;
        Operator grandParentOp =
          parentOp.getParentOperators().get(0);

        oldReduceSinkParentOps.add(parentOp);
        newParentOps.add(grandParentOp);
      }
      pos++;
    }

    // create the map-join operator
    MapJoinOperator mapJoinOp = convertJoinOpMapJoinOp(conf,
        op, leftInputJoin, baseSrc, mapAliases, mapJoinPos, noCheckOuterJoin);
    if (mapJoinOp == null) {
      return null;
    }

    // remove old parents
    for (pos = 0; pos < newParentOps.size(); pos++) {
      newParentOps.get(pos).replaceChild(oldReduceSinkParentOps.get(pos), mapJoinOp);
    }

    mapJoinOp.getParentOperators().removeAll(oldReduceSinkParentOps);
    mapJoinOp.setParentOperators(newParentOps);

    // make sure only map-joins can be performed.
    if (validateMapJoinTree) {
      validateMapJoinTypes(mapJoinOp);
    }

    // change the children of the original join operator to point to the map
    // join operator

    return mapJoinOp;
  }

  public static boolean onExpressionHasNullSafes(JoinDesc desc) {
    boolean[] nullSafes = desc.getNullSafes();
    if (nullSafes == null) {
      return false;
    }
    for (boolean nullSafe : nullSafes) {
      if (nullSafe) {
        return true;
      }
    }
    return false;
  }

  private static boolean checkFullOuterMapJoinCompatible(HiveConf hiveConf,
      JoinOperator joinOp) throws SemanticException {
    JoinDesc joinDesc = joinOp.getConf();

    // Make sure all key and value expressions are columns.
    for (Entry> mapEntry : joinDesc.getExprs().entrySet()) {
      List exprList = mapEntry.getValue();
      for (ExprNodeDesc expr : exprList) {
        if (!(expr instanceof ExprNodeColumnDesc)) {
          LOG.debug("FULL OUTER MapJoin: only column expressions are supported " + expr.toString());
          return false;
        }
      }
    }

    // Check for supported key data types.
    Byte[] order = joinDesc.getTagOrder();
    ExprNodeDesc[][] joinKeysArray = joinDesc.getJoinKeys();
    for (int i = 0; i < order.length; i++) {
      byte pos = order[i];
      ExprNodeDesc[] keyExprs = joinKeysArray[pos];
      for (ExprNodeDesc keyExpr : keyExprs) {
        TypeInfo typeInfo = keyExpr.getTypeInfo();

        // Verify we handle the key column types for an optimized table.  This is the effectively
        // the same check used in Tez HashTableLoader.
        if (!MapJoinKey.isSupportedField(typeInfo)) {
          LOG.debug("FULL OUTER MapJoin not enabled: key type {} not supported", typeInfo);
          return false;
        }
      }
    }

    if (onExpressionHasNullSafes(joinDesc)) {
      LOG.debug("FULL OUTER MapJoin not enabled: nullsafe not supported");
      return false;
    }

    boolean isVectorizationMapJoinNativeEnabled = HiveConf.getBoolVar(hiveConf,
        HiveConf.ConfVars.HIVE_VECTORIZATION_MAPJOIN_NATIVE_ENABLED);
    boolean isHybridHashJoin = HiveConf.getBoolVar(hiveConf,
        HiveConf.ConfVars.HIVEUSEHYBRIDGRACEHASHJOIN);
    if (isVectorizationMapJoinNativeEnabled && isHybridHashJoin) {
      LOG.debug("FULL OUTER MapJoin not enabled: Native Vector MapJoin and Hybrid Grace not supported");
      return false;
    }

    if (joinDesc.getResidualFilterExprs() != null &&
        joinDesc.getResidualFilterExprs().size() != 0) {
      LOG.debug("FULL OUTER MapJoin not enabled: non-equi joins not supported");
      return false;
    }

    return true;
  }

  public static boolean precheckFullOuter(HiveConf hiveConf, JoinOperator joinOp)
      throws SemanticException {
    JoinDesc joinDesc = joinOp.getConf();
    JoinCondDesc[] conds = joinDesc.getConds();

    /*
     * Are we even being asked to do a FULL OUTER JOIN?
     */
    boolean hasFullOuterJoin = false;
    for (JoinCondDesc cond : conds) {
      if (cond.getType() == JoinDesc.FULL_OUTER_JOIN) {
        hasFullOuterJoin = true;
        break;
      }
    }
    if (!hasFullOuterJoin) {
      return false;
    }

    if (conds.length > 1) {

      // No multiple condition FULL OUTER MapJoin.
      LOG.debug("FULL OUTER MapJoin not enabled: multiple JOIN conditions not supported");
      return false;
    }

    return true;
  }

  public static boolean isFullOuterMapEnabled(HiveConf hiveConf, JoinOperator joinOp)
      throws SemanticException {

    final String testMapJoinFullOuterOverrideString =
        HiveConf.getVar(hiveConf,
            HiveConf.ConfVars.HIVE_TEST_MAPJOINFULLOUTER_OVERRIDE);
    EnabledOverride mapJoinFullOuterOverride =
        EnabledOverride.NAME_MAP.get(testMapJoinFullOuterOverrideString);

    final boolean isEnabled =
        HiveConf.getBoolVar(
            hiveConf,
            HiveConf.ConfVars.HIVEMAPJOINFULLOUTER);
    switch (mapJoinFullOuterOverride) {
    case NONE:
      {
        if (!isEnabled) {
          LOG.debug("FULL OUTER MapJoin not enabled: {} is false", HiveConf.ConfVars.HIVEMAPJOINFULLOUTER.varname);
          return false;
        }
      }
      break;
    case DISABLE:
      if (LOG.isDebugEnabled()) {
        LOG.debug("FULL OUTER MapJoin not enabled: " +
            HiveConf.ConfVars.HIVE_TEST_MAPJOINFULLOUTER_OVERRIDE.varname + " is disable (" +
            " " + HiveConf.ConfVars.HIVEMAPJOINFULLOUTER.varname + " is " + isEnabled + ")");
      }
      return false;
    case ENABLE:

      // Different parts of the code may rely on this being set...
      HiveConf.setBoolVar(hiveConf,
          HiveConf.ConfVars.HIVEMAPJOINFULLOUTER, true);
      if (LOG.isDebugEnabled()) {
        LOG.debug("FULL OUTER MapJoin is enabled: " +
            HiveConf.ConfVars.HIVE_TEST_MAPJOINFULLOUTER_OVERRIDE.varname + " is enable (" +
            " " + HiveConf.ConfVars.HIVEMAPJOINFULLOUTER.varname + " is " + isEnabled + ")");
      }
      break;
    default:
      throw new RuntimeException("Unexpected vectorization enabled override " +
          mapJoinFullOuterOverride);
    }

    final String engine =
        HiveConf.getVar(
            hiveConf,
            HiveConf.ConfVars.HIVE_EXECUTION_ENGINE);
    final boolean isTezEngine = engine.equalsIgnoreCase("tez");
    if (!isTezEngine) {

      // Only Tez for now.
      LOG.debug("FULL OUTER MapJoin not enabled: Only Tez engine supported");
      return false;
    }

    /*
     * Optimized Hash Table (i.e. not old-style MR HashMap).
     */
    final boolean isOptimizedHashTableEnabled =
        HiveConf.getBoolVar(
            hiveConf,
            HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE);
    if (!isOptimizedHashTableEnabled) {
      LOG.debug("FULL OUTER MapJoin not enabled: {} is false", HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE.varname);
      return false;
    }

    boolean isCompatibleFullOuterMapJoin = checkFullOuterMapJoinCompatible(hiveConf, joinOp);
    if (!isCompatibleFullOuterMapJoin) {
      return false;
    }

    LOG.debug("FULL OUTER MapJoin enabled");
    return true;
  }

  public static boolean isFullOuterEnabledForDynamicPartitionHashJoin(HiveConf hiveConf, JoinOperator joinOp)
      throws SemanticException {
    return true;
  }

  public static MapJoinOperator convertJoinOpMapJoinOp(HiveConf hconf,
      JoinOperator op, boolean leftInputJoin, String[] baseSrc, List mapAliases,
      int mapJoinPos, boolean noCheckOuterJoin) throws SemanticException {
    return convertJoinOpMapJoinOp(hconf, op, leftInputJoin, baseSrc, mapAliases,
        mapJoinPos, noCheckOuterJoin, true);
  }

  public static MapJoinOperator convertJoinOpMapJoinOp(HiveConf hconf,
      JoinOperator op, boolean leftInputJoin, String[] baseSrc, List mapAliases,
      int mapJoinPos, boolean noCheckOuterJoin, boolean adjustParentsChildren)
          throws SemanticException {

    MapJoinDesc mapJoinDescriptor =
        getMapJoinDesc(hconf, op, leftInputJoin, baseSrc, mapAliases,
            mapJoinPos, noCheckOuterJoin, adjustParentsChildren);
    if (mapJoinDescriptor == null) {
      return null;
    }
    // reduce sink row resolver used to generate map join op
    RowSchema outputRS = op.getSchema();

    MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory.getAndMakeChild(
        op.getCompilationOpContext(), mapJoinDescriptor,
        new RowSchema(outputRS.getSignature()), op.getParentOperators());

    mapJoinOp.getConf().setReversedExprs(op.getConf().getReversedExprs());
    Map colExprMap = op.getColumnExprMap();
    mapJoinOp.setColumnExprMap(colExprMap);

    List> childOps = op.getChildOperators();
    for (Operator childOp : childOps) {
      childOp.replaceParent(op, mapJoinOp);
    }

    mapJoinOp.setPosToAliasMap(op.getPosToAliasMap());
    mapJoinOp.setChildOperators(childOps);
    op.setChildOperators(null);
    op.setParentOperators(null);

    return mapJoinOp;

  }

  private static boolean needValueIndex(int[] valueIndex) {
    for (int i = 0; i < valueIndex.length; i++) {
      if (valueIndex[i] != -i - 1) {
        return true;
      }
    }
    return false;
  }

  /**
   * convert a sortmerge join to a a map-side join.
   *
   * @param smbJoinOp
   *          join operator
   * @param bigTablePos
   *          position of the source to be read as part of map-reduce framework. All other sources
   *          are cached in memory
   * @param noCheckOuterJoin
   */
  public static MapJoinOperator convertSMBJoinToMapJoin(HiveConf hconf,
    SMBMapJoinOperator smbJoinOp, int bigTablePos, boolean noCheckOuterJoin)
    throws SemanticException {
    // Create a new map join operator
    SMBJoinDesc smbJoinDesc = smbJoinOp.getConf();
    List keyCols = smbJoinDesc.getKeys().get(Byte.valueOf((byte) 0));
    TableDesc keyTableDesc = PlanUtils.getMapJoinKeyTableDesc(hconf, PlanUtils
        .getFieldSchemasFromColumnList(keyCols, MAPJOINKEY_FIELDPREFIX));
    MapJoinDesc mapJoinDesc = new MapJoinDesc(smbJoinDesc.getKeys(),
        keyTableDesc, smbJoinDesc.getExprs(),
        smbJoinDesc.getValueTblDescs(), smbJoinDesc.getValueTblDescs(),
        smbJoinDesc.getOutputColumnNames(),
        bigTablePos, smbJoinDesc.getConds(),
        smbJoinDesc.getFilters(), smbJoinDesc.isNoOuterJoin(), smbJoinDesc.getDumpFilePrefix(),
        smbJoinDesc.getMemoryMonitorInfo(), smbJoinDesc.getInMemoryDataSize());

    mapJoinDesc.setStatistics(smbJoinDesc.getStatistics());
    mapJoinDesc.setColumnExprMap(smbJoinDesc.getColumnExprMap());

    RowSchema joinRS = smbJoinOp.getSchema();
    // The mapjoin has the same schema as the join operator
    MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory.getAndMakeChild(
        smbJoinOp.getCompilationOpContext(), mapJoinDesc, joinRS,
        new ArrayList>());

    // change the children of the original join operator to point to the map
    // join operator
    List> childOps = smbJoinOp.getChildOperators();
    for (Operator childOp : childOps) {
      childOp.replaceParent(smbJoinOp, mapJoinOp);
    }
    mapJoinOp.setChildOperators(childOps);
    smbJoinOp.setChildOperators(null);

    // change the parent of the original SMBjoin operator to point to the map
    // join operator
    List> parentOps = smbJoinOp.getParentOperators();
    for (Operator parentOp : parentOps) {
      parentOp.replaceChild(smbJoinOp, mapJoinOp);
    }
    mapJoinOp.setParentOperators(parentOps);
    smbJoinOp.setParentOperators(null);

    return mapJoinOp;
  }

  public MapJoinOperator generateMapJoinOperator(ParseContext pctx, JoinOperator op,
      int mapJoinPos) throws SemanticException {
    HiveConf hiveConf = pctx.getConf();
    boolean noCheckOuterJoin = HiveConf.getBoolVar(hiveConf,
        HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)
        && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN);

    MapJoinOperator mapJoinOp = convertMapJoin(pctx.getConf(), op,
        op.getConf().isLeftInputJoin(), op.getConf().getBaseSrc(),
        op.getConf().getMapAliases(), mapJoinPos, noCheckOuterJoin, true);
    if (mapJoinOp == null) {
      return null;
    }

    // create a dummy select to select all columns
    genSelectPlan(pctx, mapJoinOp);
    return mapJoinOp;
  }

  /**
   * Get a list of big table candidates. Only the tables in the returned set can
   * be used as big table in the join operation.
   *
   * The logic here is to scan the join condition array from left to right. If
   * see a inner join, and the bigTableCandidates is empty or the outer join
   * that we last saw is a right outer join, add both side of this inner join to
   * big table candidates only if they are not in bad position. If see a left
   * outer join, set lastSeenRightOuterJoin to false, and the bigTableCandidates
   * is empty, add the left side to it, and if the bigTableCandidates is not
   * empty, do nothing (which means the bigTableCandidates is from left side).
   * If see a right outer join, set lastSeenRightOuterJoin to true, clear the
   * bigTableCandidates, and add right side to the bigTableCandidates, it means
   * the right side of a right outer join always win. If see a full outer join,
   * return empty set immediately (no one can be the big table, can not do a
   * mapjoin).
   *
   *
   * @param condns
   * @return set of big table candidates
   */
  public static Set getBigTableCandidates(JoinCondDesc[] condns) {
    return getBigTableCandidates(condns, /* isSupportFullOuter */ false);
  }

  public static Set getBigTableCandidates(JoinCondDesc[] condns,
      boolean isSupportFullOuter) {

    Set bigTableCandidates = new HashSet();

    if (condns.length == 1) {
      JoinCondDesc condn = condns[0];
      if (condn.getType() == JoinDesc.FULL_OUTER_JOIN) {

        if (!isSupportFullOuter) {
          return new HashSet();
        }

        // FULL OUTER MapJoin must be a single condition.
        bigTableCandidates.add(condn.getLeft());
        bigTableCandidates.add(condn.getRight());
        return bigTableCandidates;
      }
    }

    boolean seenOuterJoin = false;
    Set seenPostitions = new HashSet();
    Set leftPosListOfLastRightOuterJoin = new HashSet();

    // is the outer join that we saw most recently is a right outer join?
    boolean lastSeenRightOuterJoin = false;
    for (JoinCondDesc condn : condns) {
      int joinType = condn.getType();
      if (joinType == JoinDesc.FULL_OUTER_JOIN) {
        return new HashSet();
      }

      seenPostitions.add(condn.getLeft());
      seenPostitions.add(condn.getRight());

      if (joinType == JoinDesc.LEFT_OUTER_JOIN
          || joinType == JoinDesc.LEFT_SEMI_JOIN
          || joinType == JoinDesc.ANTI_JOIN) {
        seenOuterJoin = true;
        if(bigTableCandidates.size() == 0) {
          bigTableCandidates.add(condn.getLeft());
        }

        lastSeenRightOuterJoin = false;
      } else if (joinType == JoinDesc.RIGHT_OUTER_JOIN) {
        seenOuterJoin = true;
        lastSeenRightOuterJoin = true;
        // add all except the right side to the bad positions
        leftPosListOfLastRightOuterJoin.clear();
        leftPosListOfLastRightOuterJoin.addAll(seenPostitions);
        leftPosListOfLastRightOuterJoin.remove(condn.getRight());

        bigTableCandidates.clear();
        bigTableCandidates.add(condn.getRight());
      } else if (joinType == JoinDesc.INNER_JOIN) {
        if (!seenOuterJoin || lastSeenRightOuterJoin) {
          // is the left was at the left side of a right outer join?
          if (!leftPosListOfLastRightOuterJoin.contains(condn.getLeft())) {
            bigTableCandidates.add(condn.getLeft());
          }
          // is the right was at the left side of a right outer join?
          if (!leftPosListOfLastRightOuterJoin.contains(condn.getRight())) {
            bigTableCandidates.add(condn.getRight());
          }
        }
      }
    }

    return bigTableCandidates;
  }

  /**
   * @param mapJoinPos the position of big table as determined by either hints or auto conversion.
   * @param condns the join conditions
   * @return if given mapjoin position is a feasible big table position return same else -1.
   */
  public static int checkMapJoin(int mapJoinPos, JoinCondDesc[] condns) {
    Set bigTableCandidates =
        MapJoinProcessor.getBigTableCandidates(condns, /* isSupportFullOuter */ true);

    // bigTableCandidates can never be null
    if (!bigTableCandidates.contains(mapJoinPos)) {
      return -1;
    }
    return mapJoinPos;
  }

  protected void genSelectPlan(ParseContext pctx, MapJoinOperator input) throws SemanticException {
    List> childOps = input.getChildOperators();
    input.setChildOperators(null);

    // create a dummy select - This select is needed by the walker to split the
    // mapJoin later on
    RowSchema inputRS = input.getSchema();

    ArrayList exprs = new ArrayList();
    ArrayList outputs = new ArrayList();
    List outputCols = input.getConf().getOutputColumnNames();
    ArrayList outputRS = new ArrayList();

    Map colExprMap = new HashMap();

    for (int i = 0; i < outputCols.size(); i++) {
      String internalName = outputCols.get(i);
      ColumnInfo valueInfo = inputRS.getColumnInfo(internalName);
      ExprNodeDesc colDesc = new ExprNodeColumnDesc(valueInfo.getType(), valueInfo
          .getInternalName(), valueInfo.getTabAlias(), valueInfo.getIsVirtualCol());
      exprs.add(colDesc);
      outputs.add(internalName);
      ColumnInfo newCol = new ColumnInfo(internalName, valueInfo.getType(),
              valueInfo.getTabAlias(), valueInfo.getIsVirtualCol(), valueInfo.isHiddenVirtualCol());
      newCol.setAlias(valueInfo.getAlias());
      outputRS.add(newCol);
      colExprMap.put(internalName, colDesc);
    }

    SelectDesc select = new SelectDesc(exprs, outputs, false);

    SelectOperator sel = (SelectOperator) OperatorFactory.getAndMakeChild(
        select, new RowSchema(outputRS), input);

    sel.setColumnExprMap(colExprMap);

    // Insert the select operator in between.
    sel.setChildOperators(childOps);
    for (Operator ch : childOps) {
      ch.replaceParent(input, sel);
    }
  }

  /**
   * Is it a map-side join.
   *
   * @param op
   *          join operator
   * @return -1 if it cannot be converted to a map-side join, position of the map join node
   *         otherwise
   */
  private int mapSideJoin(JoinOperator op) throws SemanticException {
    int mapJoinPos = -1;
    if (op.getConf().isMapSideJoin()) {
      int pos = 0;
      // In a map-side join, exactly one table is not present in memory.
      // The client provides the list of tables which can be cached in memory
      // via a hint.
      if (op.getConf().isLeftInputJoin()) {
        mapJoinPos = pos;
      }
      for (String src : op.getConf().getBaseSrc()) {
        if (src != null) {
          if (!op.getConf().getMapAliases().contains(src)) {
            if (mapJoinPos >= 0) {
              return -1;
            }
            mapJoinPos = pos;
          }
        }
        pos++;
      }

      // All tables are to be cached - this is not possible. In future, we can
      // support this by randomly
      // leaving some table from the list of tables to be cached
      if (mapJoinPos == -1) {
        throw new SemanticException(ErrorMsg.INVALID_MAPJOIN_HINT.getMsg(
            Arrays.toString(op.getConf().getBaseSrc())));
      }
    }

    return mapJoinPos;
  }

  /**
   * Transform the query tree. For each join, check if it is a map-side join (user specified). If
   * yes, convert it to a map-side join.
   *
   * @param pactx
   *          current parse context
   */
  @Override
  public ParseContext transform(ParseContext pactx) throws SemanticException {
    List listMapJoinOps = new ArrayList();

    // traverse all the joins and convert them if necessary
    if (pactx.getJoinOps() != null) {
      Set joinMap = new HashSet();
      Set mapJoinMap = pactx.getMapJoinOps();
      if (mapJoinMap == null) {
        mapJoinMap = new HashSet();
        pactx.setMapJoinOps(mapJoinMap);
      }

      Iterator joinCtxIter = pactx.getJoinOps().iterator();
      while (joinCtxIter.hasNext()) {
        JoinOperator joinOp = joinCtxIter.next();
        int mapJoinPos = mapSideJoin(joinOp);
        if (mapJoinPos >= 0) {
          MapJoinOperator mapJoinOp = generateMapJoinOperator(pactx, joinOp, mapJoinPos);
          listMapJoinOps.add(mapJoinOp);
          mapJoinOp.getConf().setQBJoinTreeProps(joinOp.getConf());
          mapJoinMap.add(mapJoinOp);
        } else {
          joinOp.getConf().setQBJoinTreeProps(joinOp.getConf());
          joinMap.add(joinOp);
        }
      }

      // store the new joinContext
      pactx.setJoinOps(joinMap);
    }

    // Go over the list and find if a reducer is not needed
    List> listMapJoinOpsNoRed = new ArrayList>();

    // create a walker which walks the tree in a DFS manner while maintaining
    // the operator stack.
    // The dispatcher generates the plan from the operator tree
    Map opRules = new LinkedHashMap();
    opRules.put(new RuleRegExp("R0",
      MapJoinOperator.getOperatorName() + "%"),
      getCurrentMapJoin());
    opRules.put(new RuleRegExp("R1",
      MapJoinOperator.getOperatorName() + "%.*" + FileSinkOperator.getOperatorName() + "%"),
      getMapJoinFS());
    opRules.put(new RuleRegExp("R2",
      MapJoinOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"),
      getMapJoinDefault());
    opRules.put(new RuleRegExp("R4",
      MapJoinOperator.getOperatorName() + "%.*" + UnionOperator.getOperatorName() + "%"),
      getMapJoinDefault());

    // The dispatcher fires the processor corresponding to the closest matching
    // rule and passes the context along
    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefault(), opRules, new MapJoinWalkerCtx(
        listMapJoinOpsNoRed, pactx));

    SemanticGraphWalker ogw = new GenMapRedWalker(disp);
    ArrayList topNodes = new ArrayList();
    topNodes.addAll(listMapJoinOps);
    ogw.startWalking(topNodes, null);

    pactx.setListMapJoinOpsNoReducer(listMapJoinOpsNoRed);
    return pactx;
  }

  /**
   * CurrentMapJoin.
   *
   */
  public static class CurrentMapJoin implements SemanticNodeProcessor {

    /**
     * Store the current mapjoin in the context.
     */
    @Override
    public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
        Object... nodeOutputs) throws SemanticException {

      MapJoinWalkerCtx ctx = (MapJoinWalkerCtx) procCtx;
      MapJoinOperator mapJoin = (MapJoinOperator) nd;
      if (ctx.getListRejectedMapJoins() != null && !ctx.getListRejectedMapJoins().contains(mapJoin)) {
        // for rule: MapJoin%.*MapJoin
        // have a child mapjoin. if the the current mapjoin is on a local work,
        // will put the current mapjoin in the rejected list.
        Boolean bigBranch = findGrandChildSubqueryMapjoin(ctx, mapJoin);
        if (bigBranch == null) { // no child map join
          ctx.setCurrMapJoinOp(mapJoin);
          return null;
        }
        if (bigBranch) {
          addNoReducerMapJoinToCtx(ctx, mapJoin);
        } else {
          addRejectMapJoinToCtx(ctx, mapJoin);
        }
      } else {
        ctx.setCurrMapJoinOp(mapJoin);
      }
      return null;
    }

    private Boolean findGrandChildSubqueryMapjoin(MapJoinWalkerCtx ctx, MapJoinOperator mapJoin) {
      Operator parent = mapJoin;
      while (true) {
        if (parent.getChildOperators() == null || parent.getChildOperators().size() != 1) {
          return null;
        }
        Operator ch = parent.getChildOperators().get(0);
        if (ch instanceof MapJoinOperator) {
          if (!nonSubqueryMapJoin((MapJoinOperator) ch, mapJoin)) {
            if (ch.getParentOperators().indexOf(parent) == ((MapJoinOperator) ch).getConf()
                .getPosBigTable()) {
              // not come from the local branch
              return true;
            }
          }
          return false; // not from a sub-query.
        }

        if ((ch instanceof JoinOperator) || (ch instanceof UnionOperator)
            || (ch instanceof ReduceSinkOperator) || (ch instanceof LateralViewJoinOperator)
            || (ch instanceof GroupByOperator) || (ch instanceof ScriptOperator)) {
          return null;
        }

        parent = ch;
      }
    }

    private boolean nonSubqueryMapJoin(MapJoinOperator mapJoin, MapJoinOperator parentMapJoin) {
      if (mapJoin.getParentOperators().contains(parentMapJoin)) {
        return true;
      }
      return false;
    }
  }

  private static void addNoReducerMapJoinToCtx(MapJoinWalkerCtx ctx,
      AbstractMapJoinOperator mapJoin) {
    if (ctx.getListRejectedMapJoins() != null && ctx.getListRejectedMapJoins().contains(mapJoin)) {
      return;
    }
    List> listMapJoinsNoRed = ctx
        .getListMapJoinsNoRed();
    if (listMapJoinsNoRed == null) {
      listMapJoinsNoRed = new ArrayList>();
    }
    if (!listMapJoinsNoRed.contains(mapJoin)) {
      listMapJoinsNoRed.add(mapJoin);
    }
    ctx.setListMapJoins(listMapJoinsNoRed);
  }

  private static void addRejectMapJoinToCtx(MapJoinWalkerCtx ctx,
      AbstractMapJoinOperator mapjoin) {
    // current map join is null means it has been handled by CurrentMapJoin
    // process.
    if (mapjoin == null) {
      return;
    }
    List> listRejectedMapJoins = ctx
        .getListRejectedMapJoins();
    if (listRejectedMapJoins == null) {
      listRejectedMapJoins = new ArrayList>();
    }
    if (!listRejectedMapJoins.contains(mapjoin)) {
      listRejectedMapJoins.add(mapjoin);
    }

    if (ctx.getListMapJoinsNoRed() != null && ctx.getListMapJoinsNoRed().contains(mapjoin)) {
      ctx.getListMapJoinsNoRed().remove(mapjoin);
    }

    ctx.setListRejectedMapJoins(listRejectedMapJoins);
  }

  /**
   * MapJoinFS.
   *
   */
  public static class MapJoinFS implements SemanticNodeProcessor {

    /**
     * Store the current mapjoin in a list of mapjoins followed by a filesink.
     */
    @Override
    public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
        Object... nodeOutputs) throws SemanticException {

      MapJoinWalkerCtx ctx = (MapJoinWalkerCtx) procCtx;
      AbstractMapJoinOperator mapJoin = ctx.getCurrMapJoinOp();
      List> listRejectedMapJoins = ctx
          .getListRejectedMapJoins();

      // the mapjoin has already been handled
      if ((listRejectedMapJoins != null) && (listRejectedMapJoins.contains(mapJoin))) {
        return null;
      }
      addNoReducerMapJoinToCtx(ctx, mapJoin);
      return null;
    }
  }

  /**
   * MapJoinDefault.
   *
   */
  public static class MapJoinDefault implements SemanticNodeProcessor {

    /**
     * Store the mapjoin in a rejected list.
     */
    @Override
    public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
        Object... nodeOutputs) throws SemanticException {
      MapJoinWalkerCtx ctx = (MapJoinWalkerCtx) procCtx;
      AbstractMapJoinOperator mapJoin = ctx.getCurrMapJoinOp();
      addRejectMapJoinToCtx(ctx, mapJoin);
      return null;
    }
  }

  /**
   * Default.
   *
   */
  public static class Default implements SemanticNodeProcessor {

    /**
     * Nothing to do.
     */
    @Override
    public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
        Object... nodeOutputs) throws SemanticException {
      return null;
    }
  }

  public static SemanticNodeProcessor getMapJoinFS() {
    return new MapJoinFS();
  }

  public static SemanticNodeProcessor getMapJoinDefault() {
    return new MapJoinDefault();
  }

  public static SemanticNodeProcessor getDefault() {
    return new Default();
  }

  public static SemanticNodeProcessor getCurrentMapJoin() {
    return new CurrentMapJoin();
  }

  /**
   * MapJoinWalkerCtx.
   *
   */
  public static class MapJoinWalkerCtx implements NodeProcessorCtx {

    private ParseContext pGraphContext;
    private List> listMapJoinsNoRed;
    private List> listRejectedMapJoins;
    private AbstractMapJoinOperator currMapJoinOp;

    /**
     * @param listMapJoinsNoRed
     * @param pGraphContext
     */
    public MapJoinWalkerCtx(List> listMapJoinsNoRed,
        ParseContext pGraphContext) {
      this.listMapJoinsNoRed = listMapJoinsNoRed;
      currMapJoinOp = null;
      listRejectedMapJoins = new ArrayList>();
      this.pGraphContext = pGraphContext;
    }

    /**
     * @return the listMapJoins
     */
    public List> getListMapJoinsNoRed() {
      return listMapJoinsNoRed;
    }

    /**
     * @param listMapJoinsNoRed
     *          the listMapJoins to set
     */
    public void setListMapJoins(
        List> listMapJoinsNoRed) {
      this.listMapJoinsNoRed = listMapJoinsNoRed;
    }

    /**
     * @return the currMapJoinOp
     */
    public AbstractMapJoinOperator getCurrMapJoinOp() {
      return currMapJoinOp;
    }

    /**
     * @param currMapJoinOp
     *          the currMapJoinOp to set
     */
    public void setCurrMapJoinOp(AbstractMapJoinOperator currMapJoinOp) {
      this.currMapJoinOp = currMapJoinOp;
    }

    /**
     * @return the listRejectedMapJoins
     */
    public List> getListRejectedMapJoins() {
      return listRejectedMapJoins;
    }

    /**
     * @param listRejectedMapJoins
     *          the listRejectedMapJoins to set
     */
    public void setListRejectedMapJoins(
        List> listRejectedMapJoins) {
      this.listRejectedMapJoins = listRejectedMapJoins;
    }

    public ParseContext getpGraphContext() {
      return pGraphContext;
    }

    public void setpGraphContext(ParseContext pGraphContext) {
      this.pGraphContext = pGraphContext;
    }

  }

  public static Pair, Map>> getKeys(
          boolean leftInputJoin, String[] baseSrc, JoinOperator op) {

    // Walk over all the sources (which are guaranteed to be reduce sink
    // operators).
    // The join outputs a concatenation of all the inputs.
    List oldReduceSinkParentOps =
        new ArrayList(op.getNumParent());
    if (leftInputJoin) {
      // assert mapJoinPos == 0;
      Operator parentOp = op.getParentOperators().get(0);
      assert parentOp.getParentOperators().size() == 1;
      oldReduceSinkParentOps.add((ReduceSinkOperator) parentOp);
    }

    byte pos = 0;
    for (String src : baseSrc) {
      if (src != null) {
        Operator parentOp = op.getParentOperators().get(pos);
        assert parentOp.getParentOperators().size() == 1;
        oldReduceSinkParentOps.add((ReduceSinkOperator) parentOp);
      }
      pos++;
    }

    // get the join keys from old parent ReduceSink operators
    Map> keyExprMap = new HashMap>();

    for (pos = 0; pos < op.getParentOperators().size(); pos++) {
      ReduceSinkOperator inputRS = oldReduceSinkParentOps.get(pos);
      List keyCols = inputRS.getConf().getKeyCols();
      keyExprMap.put(pos, keyCols);
    }

    return Pair.of(oldReduceSinkParentOps, keyExprMap);
  }

  public static MapJoinDesc getMapJoinDesc(HiveConf hconf,
      JoinOperator op, boolean leftInputJoin, String[] baseSrc, List mapAliases,
      int mapJoinPos, boolean noCheckOuterJoin, boolean adjustParentsChildren) throws SemanticException {
    JoinDesc desc = op.getConf();
    JoinCondDesc[] condns = desc.getConds();
    Byte[] tagOrder = desc.getTagOrder();

    // outer join cannot be performed on a table which is being cached
    if (!noCheckOuterJoin) {
      if (checkMapJoin(mapJoinPos, condns) < 0) {
        throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg());
      }
    }

    Map colExprMap = op.getColumnExprMap();
    List schema = new ArrayList(op.getSchema().getSignature());
    Map> valueExprs = op.getConf().getExprs();
    Map> newValueExprs = new HashMap>();

    Pair, Map>> pair = getKeys(leftInputJoin, baseSrc, op);
    List oldReduceSinkParentOps = pair.getLeft();
    for (Map.Entry> entry : valueExprs.entrySet()) {
      byte tag = entry.getKey();
      Operator terminal = oldReduceSinkParentOps.get(tag);

      List values = entry.getValue();
      List newValues = ExprNodeDescUtils.backtrack(values, op, terminal);
      newValueExprs.put(tag, newValues);
      for (int i = 0; i < schema.size(); i++) {
        ColumnInfo column = schema.get(i);
        if (column == null) {
          continue;
        }
        ExprNodeDesc expr = colExprMap.get(column.getInternalName());
        int index = ExprNodeDescUtils.indexOf(expr, values);
        if (index >= 0) {
          schema.set(i, null);
          if (adjustParentsChildren) {
            // Since we remove reduce sink parents, replace original expressions
            colExprMap.put(column.getInternalName(), newValues.get(index));
          }
        }
      }
    }

    // rewrite value index for mapjoin
    Map valueIndices = new HashMap();

    // get the join keys from old parent ReduceSink operators
    Map> keyExprMap = pair.getRight();

    if (!adjustParentsChildren) {
      // Since we did not remove reduce sink parents, keep the original value expressions
      newValueExprs = valueExprs;

      // Join key exprs are represented in terms of the original table columns,
      // we need to convert these to the generated column names we can see in the Join operator
      Map> newKeyExprMap = new HashMap>();
      for (Map.Entry> mapEntry : keyExprMap.entrySet()) {
        Byte pos = mapEntry.getKey();
        ReduceSinkOperator rsParent = oldReduceSinkParentOps.get(pos.byteValue());
        List keyExprList =
            ExprNodeDescUtils.resolveJoinKeysAsRSColumns(mapEntry.getValue(), rsParent);
        if (keyExprList == null) {
          throw new SemanticException("Error resolving join keys");
        }
        newKeyExprMap.put(pos, keyExprList);
      }
      keyExprMap = newKeyExprMap;
    }

    // construct valueTableDescs and valueFilteredTableDescs
    List valueTableDescs = new ArrayList();
    List valueFilteredTableDescs = new ArrayList();
    int[][] filterMap = desc.getFilterMap();
    for (byte pos = 0; pos < op.getParentOperators().size(); pos++) {
      List valueCols = newValueExprs.get(pos);
      if (pos != mapJoinPos) {
        // remove values in key exprs for value table schema
        // value expression for hashsink will be modified in
        // LocalMapJoinProcessor
        int[] valueIndex = new int[valueCols.size()];
        List valueColsInValueExpr = new ArrayList();
        for (int i = 0; i < valueIndex.length; i++) {
          ExprNodeDesc expr = valueCols.get(i);
          int kindex = ExprNodeDescUtils.indexOf(expr, keyExprMap.get(pos));
          if (kindex >= 0) {
            valueIndex[i] = kindex;
          } else {
            valueIndex[i] = -valueColsInValueExpr.size() - 1;
            valueColsInValueExpr.add(expr);
          }
        }
        if (needValueIndex(valueIndex)) {
          valueIndices.put(pos, valueIndex);
        }
        valueCols = valueColsInValueExpr;
      }
      // deep copy expr node desc
      List valueFilteredCols = ExprNodeDescUtils.clone(valueCols);
      if (filterMap != null && filterMap[pos] != null && pos != mapJoinPos) {
        ExprNodeColumnDesc isFilterDesc =
            new ExprNodeColumnDesc(
                TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.SMALLINT_TYPE_NAME), "filter",
                "filter", false);
        valueFilteredCols.add(isFilterDesc);
      }

      TableDesc valueTableDesc =
          PlanUtils.getMapJoinValueTableDesc(PlanUtils.getFieldSchemasFromColumnList(valueCols,
              "mapjoinvalue"));
      TableDesc valueFilteredTableDesc =
          PlanUtils.getMapJoinValueTableDesc(PlanUtils.getFieldSchemasFromColumnList(
              valueFilteredCols, "mapjoinvalue"));

      valueTableDescs.add(valueTableDesc);
      valueFilteredTableDescs.add(valueFilteredTableDesc);
    }

    Map> filters = desc.getFilters();
    if(adjustParentsChildren) {
      // backtrack and update filter expressions only if RS is to be removed
      Map> newFilters = new HashMap>();
      for (Map.Entry> entry : filters.entrySet()) {
        byte srcTag = entry.getKey();
        List filter = entry.getValue();

        Operator terminal = oldReduceSinkParentOps.get(srcTag);
        newFilters.put(srcTag, ExprNodeDescUtils.backtrack(filter, op, terminal));
      }
      desc.setFilters(filters = newFilters);
    }

    // create dumpfile prefix needed to create descriptor
    String dumpFilePrefix = "";
    if (mapAliases != null) {
      for (String mapAlias : mapAliases) {
        dumpFilePrefix = dumpFilePrefix + mapAlias;
      }
      dumpFilePrefix = dumpFilePrefix + "-" + PlanUtils.getCountForMapJoinDumpFilePrefix();
    } else {
      dumpFilePrefix = "mapfile" + PlanUtils.getCountForMapJoinDumpFilePrefix();
    }

    List keyCols = keyExprMap.get((byte) mapJoinPos);
    if (keyCols == null) {
      return null;
    }

    List outputColumnNames = op.getConf().getOutputColumnNames();
    TableDesc keyTableDesc =
        PlanUtils.getMapJoinKeyTableDesc(hconf,
            PlanUtils.getFieldSchemasFromColumnList(keyCols, MAPJOINKEY_FIELDPREFIX));
    JoinCondDesc[] joinCondns = op.getConf().getConds();
    MapJoinDesc mapJoinDescriptor =
        new MapJoinDesc(keyExprMap, keyTableDesc, newValueExprs, valueTableDescs,
            valueFilteredTableDescs, outputColumnNames, mapJoinPos, joinCondns, filters,
            op.getConf().getNoOuterJoin(), dumpFilePrefix,
            op.getConf().getMemoryMonitorInfo(), op.getConf().getInMemoryDataSize());
    mapJoinDescriptor.setStatistics(op.getConf().getStatistics());
    mapJoinDescriptor.setTagOrder(tagOrder);
    mapJoinDescriptor.setNullSafes(desc.getNullSafes());
    mapJoinDescriptor.setFilterMap(desc.getFilterMap());
    mapJoinDescriptor.setResidualFilterExprs(desc.getResidualFilterExprs());
    mapJoinDescriptor.setColumnExprMap(colExprMap);
    if (!valueIndices.isEmpty()) {
      mapJoinDescriptor.setValueIndices(valueIndices);
    }

    return mapJoinDescriptor;
  }

  public static MapJoinDesc getMapJoinDesc(HiveConf hconf,
      JoinOperator op, boolean leftInputJoin, String[] baseSrc, List mapAliases,
      int mapJoinPos, boolean noCheckOuterJoin) throws SemanticException {
    return getMapJoinDesc(hconf, op, leftInputJoin, baseSrc,
        mapAliases, mapJoinPos, noCheckOuterJoin, true);
  }
}




© 2015 - 2024 Weber Informatics LLC | Privacy Policy