http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/HiveIndexedInputFormat.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/HiveIndexedInputFormat.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/HiveIndexedInputFormat.java
deleted file mode 100644
index b5bddec..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/HiveIndexedInputFormat.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * 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.index;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.Arrays;
-import java.util.Collection;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
-import org.apache.hadoop.hive.ql.io.HiveInputFormat;
-import org.apache.hadoop.hive.ql.io.IOPrepareCache;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.PartitionDesc;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-
-/**
- * Input format for doing queries that use indexes.
- * Uses a blockfilter file to specify the blocks to query.
- */
-public class HiveIndexedInputFormat extends HiveInputFormat {
-  public static final Logger l4j = 
LoggerFactory.getLogger("HiveIndexInputFormat");
-  private final String indexFile;
-
-  public HiveIndexedInputFormat() {
-    super();
-    indexFile = "hive.index.blockfilter.file";
-  }
-
-  public HiveIndexedInputFormat(String indexFileName) {
-    indexFile = indexFileName;
-  }
-
-  public InputSplit[] doGetSplits(JobConf job, int numSplits) throws 
IOException {
-
-    super.init(job);
-
-    Path[] dirs = FileInputFormat.getInputPaths(job);
-    if (dirs.length == 0) {
-      throw new IOException("No input paths specified in job");
-    }
-    JobConf newjob = new JobConf(job);
-    ArrayList<InputSplit> result = new ArrayList<InputSplit>();
-
-    // for each dir, get the InputFormat, and do getSplits.
-    PartitionDesc part;
-    for (Path dir : dirs) {
-      part = HiveFileFormatUtils
-          .getFromPathRecursively(pathToPartitionInfo, dir,
-              IOPrepareCache.get().allocatePartitionDescMap(), true);
-      // create a new InputFormat instance if this is the first time to see 
this
-      // class
-      Class inputFormatClass = part.getInputFileFormatClass();
-      InputFormat inputFormat = getInputFormatFromCache(inputFormatClass, job);
-
-      try {
-        Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), newjob);
-      } catch (HiveException e) {
-        throw new IOException(e);
-      }
-
-      FileInputFormat.setInputPaths(newjob, dir);
-      newjob.setInputFormat(inputFormat.getClass());
-      InputSplit[] iss = inputFormat.getSplits(newjob, numSplits / 
dirs.length);
-      for (InputSplit is : iss) {
-        result.add(new HiveInputSplit(is, inputFormatClass.getName()));
-      }
-    }
-    return result.toArray(new HiveInputSplit[result.size()]);
-  }
-
-  public static List<String> getIndexFiles(String indexFileStr) {
-    // tokenize and store string of form (path,)+
-    if (indexFileStr == null) {
-      return null;
-    }
-    String[] chunks = indexFileStr.split(",");
-    return Arrays.asList(chunks);
-  }
-
-  @Override
-  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException 
{
-    String indexFileStr = job.get(indexFile);
-    l4j.info("index_file is " + indexFileStr);
-    List<String> indexFiles = getIndexFiles(indexFileStr);
-
-    HiveIndexResult hiveIndexResult = null;
-    if (indexFiles != null) {
-      boolean first = true;
-      StringBuilder newInputPaths = new StringBuilder();
-      try {
-        hiveIndexResult = new HiveIndexResult(indexFiles, job);
-      } catch (HiveException e) {
-        l4j.error("Unable to read index..");
-        throw new IOException(e);
-      }
-
-      Set<String> inputFiles = hiveIndexResult.buckets.keySet();
-      if (inputFiles == null || inputFiles.size() <= 0) {
-        // return empty splits if index results were empty
-        return new InputSplit[0];
-      }
-      Iterator<String> iter = inputFiles.iterator();
-      while(iter.hasNext()) {
-        String path = iter.next();
-        if (path.trim().equalsIgnoreCase("")) {
-          continue;
-        }
-        if (!first) {
-          newInputPaths.append(",");
-        } else {
-          first = false;
-        }
-        newInputPaths.append(path);
-      }
-      FileInputFormat.setInputPaths(job, newInputPaths.toString());
-    } else {
-      return super.getSplits(job, numSplits);
-    }
-
-    HiveInputSplit[] splits = (HiveInputSplit[]) this.doGetSplits(job, 
numSplits);
-
-    long maxInputSize = HiveConf.getLongVar(job, 
ConfVars.HIVE_INDEX_COMPACT_QUERY_MAX_SIZE);
-    if (maxInputSize < 0) {
-      maxInputSize=Long.MAX_VALUE;
-    }
-
-    SplitFilter filter = new SplitFilter(hiveIndexResult, maxInputSize);
-    Collection<HiveInputSplit> newSplits = filter.filter(splits);
-
-    return newSplits.toArray(new FileSplit[newSplits.size()]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/IndexMetadataChangeTask.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexMetadataChangeTask.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexMetadataChangeTask.java
deleted file mode 100644
index 9e714e4..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexMetadataChangeTask.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.index;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.ql.DriverContext;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.metadata.Partition;
-import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.plan.api.StageType;
-
-public class IndexMetadataChangeTask extends Task<IndexMetadataChangeWork>{
-
-  private static final long serialVersionUID = 1L;
-
-  @Override
-  protected int execute(DriverContext driverContext) {
-
-    try {
-      Hive db = Hive.get(conf);
-      IndexMetadataChangeWork work = this.getWork();
-      String tblName = work.getIndexTbl();
-      Table tbl = db.getTable(work.getDbName(), tblName);
-      if (tbl == null ) {
-        console.printError("Index table can not be null.");
-        return 1;
-      }
-
-      if (!tbl.getTableType().equals(TableType.INDEX_TABLE)) {
-        console.printError("Table " + tbl.getTableName() + " not specified.");
-        return 1;
-      }
-
-      if (tbl.isPartitioned() && work.getPartSpec() == null) {
-        console.printError("Index table is partitioned, but no partition 
specified.");
-        return 1;
-      }
-
-      if (work.getPartSpec() != null) {
-        Partition part = db.getPartition(tbl, work.getPartSpec(), false);
-        if (part == null) {
-          console.printError("Partition " +
-              Warehouse.makePartName(work.getPartSpec(), false).toString()
-              + " does not exist.");
-          return 1;
-        }
-
-        Path path = part.getDataLocation();
-        FileSystem fs = path.getFileSystem(conf);
-        FileStatus fstat = fs.getFileStatus(path);
-
-        part.getParameters().put(HiveIndex.INDEX_TABLE_CREATETIME, 
Long.toString(fstat.getModificationTime()));
-        db.alterPartition(tbl.getTableName(), part, null);
-      } else {
-        Path url = new Path(tbl.getPath().toString());
-        FileSystem fs = url.getFileSystem(conf);
-        FileStatus fstat = fs.getFileStatus(url);
-        tbl.getParameters().put(HiveIndex.INDEX_TABLE_CREATETIME, 
Long.toString(fstat.getModificationTime()));
-        db.alterTable(tbl, null);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-      console.printError("Error changing index table/partition metadata "
-          + e.getMessage());
-      return 1;
-    }
-    return 0;
-  }
-
-  @Override
-  public String getName() {
-    return IndexMetadataChangeTask.class.getSimpleName();
-  }
-
-  @Override
-  public StageType getType() {
-    return StageType.DDL;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/IndexMetadataChangeWork.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexMetadataChangeWork.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexMetadataChangeWork.java
deleted file mode 100644
index 6d77ea4..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexMetadataChangeWork.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.index;
-
-import java.io.Serializable;
-import java.util.HashMap;
-
-public class IndexMetadataChangeWork implements Serializable {
-
-  private static final long serialVersionUID = 1L;
-
-  private HashMap<String, String> partSpec;
-  private String indexTbl;
-  private String dbName;
-
-  public IndexMetadataChangeWork() {
-  }
-
-  public IndexMetadataChangeWork(HashMap<String, String> partSpec,
-      String indexTbl, String dbName) {
-    super();
-    this.partSpec = partSpec;
-    this.indexTbl = indexTbl;
-    this.dbName = dbName;
-  }
-
-  public HashMap<String, String> getPartSpec() {
-    return partSpec;
-  }
-
-  public void setPartSpec(HashMap<String, String> partSpec) {
-    this.partSpec = partSpec;
-  }
-
-  public String getIndexTbl() {
-    return indexTbl;
-  }
-
-  public void setIndexTbl(String indexTbl) {
-    this.indexTbl = indexTbl;
-  }
-
-  public String getDbName() {
-    return dbName;
-  }
-
-  public void setDbName(String dbName) {
-    this.dbName = dbName;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java
index 7476036..6a3f3b4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java
@@ -61,7 +61,10 @@ import 
org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseCompare;
  * Currently, it only supports pure conjunctions over binary expressions
  * comparing a column reference with a constant value.  It is assumed
  * that all column aliases encountered refer to the same table.
+ *
+ * @deprecated kept only because some storagehandlers are using it internally
  */
+@Deprecated
 public class IndexPredicateAnalyzer {
 
   private final Set<String> udfNames;

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/IndexResult.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexResult.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexResult.java
deleted file mode 100644
index e8f2daf..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexResult.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.index;
-
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.mapred.FileSplit;
-
-public interface IndexResult {
-  boolean contains(FileSplit split) throws HiveException;
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/IndexSearchCondition.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexSearchCondition.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexSearchCondition.java
index 15cb1f7..3985246 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexSearchCondition.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexSearchCondition.java
@@ -25,7 +25,9 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
  * IndexSearchCondition represents an individual search condition
  * found by {@link IndexPredicateAnalyzer}.
  *
+ * @deprecated kept only because some storagehandlers are using it internally
  */
+@Deprecated
 public class IndexSearchCondition
 {
   private ExprNodeColumnDesc columnDesc;
@@ -56,7 +58,7 @@ public class IndexSearchCondition
    * @param constantDesc constant value to search for
    *
    * @param indexExpr the comparison expression for the index
-   * 
+   *
    * @param originalExpr the original comparison expression
    */
   public IndexSearchCondition(

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/SplitFilter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/index/SplitFilter.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/SplitFilter.java
deleted file mode 100644
index c51dec6..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/SplitFilter.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.index;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterables;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.List;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.io.HiveInputFormat.HiveInputSplit;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.mapred.FileSplit;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public final class SplitFilter {
-  public static final Logger LOG = LoggerFactory.getLogger(SplitFilter.class);
-
-  private final IndexResult indexResult;
-  private final long maxInputSize;
-
-  public SplitFilter(IndexResult indexResult, long maxInputSize) {
-    this.indexResult = indexResult;
-    this.maxInputSize = maxInputSize;
-  }
-
-  public List<HiveInputSplit> filter(HiveInputSplit[] splits) throws 
IOException {
-    long sumSplitLengths = 0;
-    List<HiveInputSplit> newSplits = new ArrayList<>();
-
-    Arrays.sort(splits, new HiveInputSplitComparator());
-
-    for (HiveInputSplit split : splits) {
-      LOG.info("split start : " + split.getStart());
-      LOG.info("split end : " + (split.getStart() + split.getLength()));
-
-      try {
-        if (indexResult.contains(split)) {
-          HiveInputSplit newSplit = split;
-          if (isAdjustmentRequired(newSplits, split)) {
-            newSplit = adjustSplit(split);
-          }
-          sumSplitLengths += newSplit.getLength();
-          if (sumSplitLengths > maxInputSize) {
-            String messageTemplate = "Size of data to read during a 
compact-index-based query " +
-                "exceeded the maximum of %d set in %s";
-            throw new IOException(String.format(messageTemplate, maxInputSize,
-                HiveConf.ConfVars.HIVE_INDEX_COMPACT_QUERY_MAX_SIZE.varname));
-          }
-          newSplits.add(newSplit);
-        }
-      } catch (HiveException e) {
-        throw new RuntimeException("Unable to get metadata for input table 
split " +
-            split.getPath(), e);
-      }
-    }
-    LOG.info("Number of input splits: {}, new input splits: {}, sum of split 
lengths: {}",
-        splits.length, newSplits.size(), sumSplitLengths);
-    return newSplits;
-  }
-
-  private boolean isAdjustmentRequired(List<HiveInputSplit> newSplits, 
HiveInputSplit split) {
-    return (split.inputFormatClassName().contains("RCFile") ||
-        split.inputFormatClassName().contains("SequenceFile")) && 
split.getStart() > 0 &&
-        !doesOverlap(newSplits, split.getPath(), 
adjustStart(split.getStart()));
-  }
-
-  private boolean doesOverlap(List<HiveInputSplit> newSplits, Path path, long 
start) {
-    if (newSplits.isEmpty()) {
-      return false;
-    }
-    HiveInputSplit lastSplit = Iterables.getLast(newSplits);
-    if (lastSplit.getPath().equals(path)) {
-      return lastSplit.getStart() + lastSplit.getLength() > start;
-    }
-    return false;
-  }
-
-  private long adjustStart(long start) {
-    return start > SequenceFile.SYNC_INTERVAL ? start - 
SequenceFile.SYNC_INTERVAL : 0;
-  }
-
-  private HiveInputSplit adjustSplit(HiveInputSplit split) throws IOException {
-    long adjustedStart = adjustStart(split.getStart());
-    return new HiveInputSplit(new FileSplit(split.getPath(), adjustedStart,
-        split.getStart() - adjustedStart + split.getLength(), 
split.getLocations()),
-        split.inputFormatClassName());
-  }
-
-  @VisibleForTesting
-  static final class HiveInputSplitComparator implements 
Comparator<HiveInputSplit> {
-    @Override
-    public int compare(HiveInputSplit o1, HiveInputSplit o2) {
-      int pathCompare = comparePath(o1.getPath(), o2.getPath());
-      if (pathCompare != 0) {
-        return pathCompare;
-      }
-      return Long.compare(o1.getStart(), o2.getStart());
-    }
-
-    private int comparePath(Path p1, Path p2) {
-      return p1.compareTo(p2);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/TableBasedIndexHandler.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/TableBasedIndexHandler.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/TableBasedIndexHandler.java
deleted file mode 100644
index d861522..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/TableBasedIndexHandler.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * 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.index;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Set;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.HiveUtils;
-import org.apache.hadoop.hive.ql.metadata.Partition;
-import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.plan.PartitionDesc;
-import org.apache.hadoop.hive.ql.plan.TableDesc;
-import org.apache.hadoop.hive.ql.session.LineageState;
-
-/**
- * Index handler for indexes that use tables to store indexes.
- */
-public abstract class TableBasedIndexHandler extends AbstractIndexHandler {
-  protected Configuration configuration;
-
-  @Override
-  public List<Task<?>> generateIndexBuildTaskList(
-      org.apache.hadoop.hive.ql.metadata.Table baseTbl,
-      org.apache.hadoop.hive.metastore.api.Index index,
-      List<Partition> indexTblPartitions, List<Partition> baseTblPartitions,
-      org.apache.hadoop.hive.ql.metadata.Table indexTbl,
-      Set<ReadEntity> inputs, Set<WriteEntity> outputs,
-      LineageState lineageState) throws HiveException {
-    try {
-
-      TableDesc desc = Utilities.getTableDesc(indexTbl);
-
-      List<Partition> newBaseTblPartitions = new ArrayList<Partition>();
-
-      List<Task<?>> indexBuilderTasks = new ArrayList<Task<?>>();
-
-      if (!baseTbl.isPartitioned()) {
-        // the table does not have any partition, then create index for the
-        // whole table
-        Task<?> indexBuilder = getIndexBuilderMapRedTask(inputs, outputs, 
index, false,
-            new PartitionDesc(desc, null), indexTbl.getTableName(),
-            new PartitionDesc(Utilities.getTableDesc(baseTbl), null),
-            baseTbl.getTableName(), indexTbl.getDbName(), lineageState);
-        indexBuilderTasks.add(indexBuilder);
-      } else {
-
-        // check whether the index table partitions are still exists in base
-        // table
-        for (int i = 0; i < indexTblPartitions.size(); i++) {
-          Partition indexPart = indexTblPartitions.get(i);
-          Partition basePart = null;
-          for (int j = 0; j < baseTblPartitions.size(); j++) {
-            if 
(baseTblPartitions.get(j).getName().equals(indexPart.getName())) {
-              basePart = baseTblPartitions.get(j);
-              newBaseTblPartitions.add(baseTblPartitions.get(j));
-              break;
-            }
-          }
-          if (basePart == null) {
-            throw new RuntimeException(
-                "Partitions of base table and index table are inconsistent.");
-          }
-          // for each partition, spawn a map reduce task.
-          Task<?> indexBuilder = getIndexBuilderMapRedTask(inputs, outputs, 
index, true,
-              new PartitionDesc(indexPart), indexTbl.getTableName(),
-              new PartitionDesc(basePart), baseTbl.getTableName(), 
indexTbl.getDbName(),
-              lineageState);
-          indexBuilderTasks.add(indexBuilder);
-        }
-      }
-      return indexBuilderTasks;
-    } catch (Exception e) {
-      throw new SemanticException(e);
-    }
-  }
-
-  protected Task<?> getIndexBuilderMapRedTask(Set<ReadEntity> inputs, 
Set<WriteEntity> outputs,
-      Index index, boolean partitioned,
-      PartitionDesc indexTblPartDesc, String indexTableName,
-      PartitionDesc baseTablePartDesc, String baseTableName, String dbName,
-      LineageState lineageState) throws HiveException {
-    return getIndexBuilderMapRedTask(inputs, outputs, index.getSd().getCols(),
-        partitioned, indexTblPartDesc, indexTableName, baseTablePartDesc, 
baseTableName, dbName,
-        lineageState);
-  }
-
-  protected Task<?> getIndexBuilderMapRedTask(Set<ReadEntity> inputs, 
Set<WriteEntity> outputs,
-      List<FieldSchema> indexField, boolean partitioned,
-      PartitionDesc indexTblPartDesc, String indexTableName,
-      PartitionDesc baseTablePartDesc, String baseTableName, String dbName,
-      LineageState lineageState) throws HiveException {
-    return null;
-  }
-
-  protected List<String> getPartKVPairStringArray(
-      LinkedHashMap<String, String> partSpec) {
-    List<String> ret = new ArrayList<String>(partSpec.size());
-    Iterator<Entry<String, String>> iter = partSpec.entrySet().iterator();
-    while (iter.hasNext()) {
-      StringBuilder sb = new StringBuilder();
-      Entry<String, String> p = iter.next();
-      sb.append(HiveUtils.unparseIdentifier(p.getKey()));
-      sb.append(" = ");
-      sb.append("'");
-      sb.append(HiveUtils.escapeString(p.getValue()));
-      sb.append("'");
-      ret.add(sb.toString());
-    }
-    return ret;
-  }
-
-  @Override
-  public boolean usesIndexTable() {
-    return true;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return configuration;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.configuration = conf;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapIndexHandler.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapIndexHandler.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapIndexHandler.java
deleted file mode 100644
index 62db4db..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapIndexHandler.java
+++ /dev/null
@@ -1,312 +0,0 @@
-/*
- * 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.index.bitmap;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.Driver;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.index.HiveIndexQueryContext;
-import org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat;
-import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer;
-import org.apache.hadoop.hive.ql.index.IndexSearchCondition;
-import org.apache.hadoop.hive.ql.index.TableBasedIndexHandler;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.HiveUtils;
-import org.apache.hadoop.hive.ql.metadata.Partition;
-import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
-import org.apache.hadoop.hive.ql.optimizer.IndexUtils;
-import org.apache.hadoop.hive.ql.parse.ParseContext;
-import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.plan.PartitionDesc;
-import org.apache.hadoop.hive.ql.session.LineageState;
-import org.apache.hadoop.hive.ql.stats.StatsUtils;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
-
-/**
- * Index handler for the bitmap index. Bitmap index uses an EWAH-compressed
- * bitmap to represent the values in a table.
- */
-public class BitmapIndexHandler extends TableBasedIndexHandler {
-
-  private Configuration configuration;
-  private static final Logger LOG = 
LoggerFactory.getLogger(BitmapIndexHandler.class.getName());
-
-  @Override
-  public void generateIndexQuery(List<Index> indexes, ExprNodeDesc predicate,
-    ParseContext pctx, HiveIndexQueryContext queryContext) {
-
-    Map<Index, ExprNodeDesc> indexPredicates  = decomposePredicate(
-                                                              predicate,
-                                                              indexes,
-                                                              queryContext);
-
-    if (indexPredicates == null) {
-      LOG.info("No decomposed predicate found");
-      queryContext.setQueryTasks(null);
-      return; // abort if we couldn't pull out anything from the predicate
-    }
-
-    List<BitmapInnerQuery> iqs = new 
ArrayList<BitmapInnerQuery>(indexes.size());
-    int i = 0;
-    for (Index index : indexes) {
-      ExprNodeDesc indexPredicate = indexPredicates.get(index);
-      if (indexPredicate != null) {
-        iqs.add(new BitmapInnerQuery(
-              index.getIndexTableName(),
-              indexPredicate,
-              "ind" + i++));
-      }
-    }
-    // setup TableScanOperator to change input format for original query
-    queryContext.setIndexInputFormat(HiveIndexedInputFormat.class.getName());
-
-    // Build reentrant QL for index query
-    StringBuilder qlCommand = new StringBuilder("INSERT OVERWRITE DIRECTORY ");
-
-    String tmpFile = pctx.getContext().getMRTmpPath().toUri().toString();
-    qlCommand.append( "\"" + tmpFile + "\" ");            // QL includes " 
around file name
-    qlCommand.append("SELECT bucketname AS `_bucketname` , COLLECT_SET(offset) 
AS `_offsets` FROM ");
-    qlCommand.append("(SELECT `_bucketname` AS bucketname , `_offset` AS 
offset FROM ");
-
-
-    BitmapQuery head = iqs.get(0);
-    for ( i = 1; i < iqs.size(); i++) {
-      head = new BitmapOuterQuery("oind"+i, head, iqs.get(i));
-    }
-    qlCommand.append(head.toString());
-    qlCommand.append(" WHERE NOT EWAH_BITMAP_EMPTY(" + head.getAlias() + 
".`_bitmaps`) ) tmp_index GROUP BY bucketname");
-
-    // generate tasks from index query string
-    LOG.info("Generating tasks for re-entrant QL query: " + 
qlCommand.toString());
-    HiveConf queryConf = new HiveConf(pctx.getConf(), 
BitmapIndexHandler.class);
-    HiveConf.setBoolVar(queryConf, HiveConf.ConfVars.COMPRESSRESULT, false);
-    Driver driver = new Driver(queryConf, 
pctx.getQueryState().getLineageState());
-    driver.compile(qlCommand.toString(), false);
-
-    queryContext.setIndexIntermediateFile(tmpFile);
-    queryContext.addAdditionalSemanticInputs(driver.getPlan().getInputs());
-    queryContext.setQueryTasks(driver.getPlan().getRootTasks());
-  }
-
-  /**
-   * Split the predicate into the piece we can deal with (pushed), and the one 
we can't (residual)
-   * @param predicate
-   * @param index
-   * @return
-   */
-  private Map<Index, ExprNodeDesc> decomposePredicate(ExprNodeDesc predicate, 
List<Index> indexes,
-      HiveIndexQueryContext queryContext) {
-
-    Map<Index, ExprNodeDesc> indexPredicates = new HashMap<Index, 
ExprNodeDesc>();
-    // compute overall residual
-    IndexPredicateAnalyzer analyzer = getIndexPredicateAnalyzer(indexes, 
queryContext.getQueryPartitions());
-    List<IndexSearchCondition> searchConditions = new 
ArrayList<IndexSearchCondition>();
-    ExprNodeDesc residualPredicate = analyzer.analyzePredicate(predicate, 
searchConditions);
-    // pass residual predicate back out for further processing
-    queryContext.setResidualPredicate(residualPredicate);
-
-    if (searchConditions.size() == 0) {
-      return null;
-    }
-
-    for (Index index : indexes) {
-      ArrayList<Index> in = new ArrayList<Index>(1);
-      in.add(index);
-      analyzer = getIndexPredicateAnalyzer(in, 
queryContext.getQueryPartitions());
-      searchConditions = new ArrayList<IndexSearchCondition>();
-      // split predicate into pushed (what we can handle), and residual (what 
we can't handle)
-      // pushed predicate from translateSearchConditions is stored for the 
current index
-      // This ensures that we apply all possible predicates to each index
-      analyzer.analyzePredicate(predicate, searchConditions);
-      if (searchConditions.size() == 0) {
-        indexPredicates.put(index, null);
-      } else {
-        indexPredicates.put(index, 
analyzer.translateSearchConditions(searchConditions));
-      }
-    }
-
-    return indexPredicates;
-  }
-
-  /**
-   * Instantiate a new predicate analyzer suitable for determining
-   * whether we can use an index, based on rules for indexes in
-   * WHERE clauses that we support
-   *
-   * @return preconfigured predicate analyzer for WHERE queries
-   */
-  private IndexPredicateAnalyzer getIndexPredicateAnalyzer(List<Index> 
indexes, Set<Partition> queryPartitions)  {
-    IndexPredicateAnalyzer analyzer = new IndexPredicateAnalyzer();
-
-    analyzer.addComparisonOp(GenericUDFOPEqual.class.getName());
-    analyzer.addComparisonOp(GenericUDFOPLessThan.class.getName());
-    analyzer.addComparisonOp(GenericUDFOPEqualOrLessThan.class.getName());
-    analyzer.addComparisonOp(GenericUDFOPGreaterThan.class.getName());
-    analyzer.addComparisonOp(GenericUDFOPEqualOrGreaterThan.class.getName());
-
-    // only return results for columns in the list of indexes
-    for (Index index : indexes) {
-      List<FieldSchema> columnSchemas = index.getSd().getCols();
-      for (FieldSchema column : columnSchemas) {
-        analyzer.allowColumnName(column.getName());
-      }
-    }
-
-    // partitioned columns are treated as if they have indexes so that the 
partitions
-    // are used during the index query generation
-    for (Partition part : queryPartitions) {
-      if (part.getSpec().isEmpty()) {
-        continue; // empty partitions are from whole tables, so we don't want 
to add them in
-      }
-      for (String column : part.getSpec().keySet()) {
-        analyzer.allowColumnName(column);
-      }
-    }
-
-    return analyzer;
-  }
-
-  @Override
-  public void analyzeIndexDefinition(Table baseTable, Index index,
-      Table indexTable) throws HiveException {
-    StorageDescriptor storageDesc = index.getSd();
-    if (this.usesIndexTable() && indexTable != null) {
-      StorageDescriptor indexTableSd = storageDesc.deepCopy();
-      List<FieldSchema> indexTblCols = indexTableSd.getCols();
-      FieldSchema bucketFileName = new FieldSchema("_bucketname", "string", 
"");
-      indexTblCols.add(bucketFileName);
-      FieldSchema offSets = new FieldSchema("_offset", "bigint", "");
-      indexTblCols.add(offSets);
-      FieldSchema bitmaps = new FieldSchema("_bitmaps", "array<bigint>", "");
-      indexTblCols.add(bitmaps);
-      indexTable.setSd(indexTableSd);
-    }
-  }
-
-  @Override
-  protected Task<?> getIndexBuilderMapRedTask(Set<ReadEntity> inputs, 
Set<WriteEntity> outputs,
-      List<FieldSchema> indexField, boolean partitioned,
-      PartitionDesc indexTblPartDesc, String indexTableName,
-      PartitionDesc baseTablePartDesc, String baseTableName, String dbName,
-      LineageState lineageState) throws HiveException {
-
-    HiveConf builderConf = new HiveConf(getConf(), BitmapIndexHandler.class);
-    HiveConf.setBoolVar(builderConf, HiveConf.ConfVars.HIVEROWOFFSET, true);
-
-    String indexCols = 
HiveUtils.getUnparsedColumnNamesFromFieldSchema(indexField);
-
-    //form a new insert overwrite query.
-    StringBuilder command= new StringBuilder();
-    LinkedHashMap<String, String> partSpec = indexTblPartDesc.getPartSpec();
-
-    String fullIndexTableName = 
StatsUtils.getFullyQualifiedTableName(HiveUtils.unparseIdentifier(dbName),
-        HiveUtils.unparseIdentifier(indexTableName));
-    command.append("INSERT OVERWRITE TABLE " + fullIndexTableName);
-    if (partitioned && indexTblPartDesc != null) {
-      command.append(" PARTITION ( ");
-      List<String> ret = getPartKVPairStringArray(partSpec);
-      for (int i = 0; i < ret.size(); i++) {
-        String partKV = ret.get(i);
-        command.append(partKV);
-        if (i < ret.size() - 1) {
-          command.append(",");
-        }
-      }
-      command.append(" ) ");
-    }
-
-    String fullBaseTableName = 
StatsUtils.getFullyQualifiedTableName(HiveUtils.unparseIdentifier(dbName),
-        HiveUtils.unparseIdentifier(baseTableName));
-    command.append(" SELECT ");
-    command.append(indexCols);
-    command.append(",");
-    command.append(VirtualColumn.FILENAME.getName());
-    command.append(",");
-    command.append(VirtualColumn.BLOCKOFFSET.getName());
-    command.append(",");
-    command.append("EWAH_BITMAP(");
-    command.append(VirtualColumn.ROWOFFSET.getName());
-    command.append(")");
-    command.append(" FROM " + fullBaseTableName);
-    LinkedHashMap<String, String> basePartSpec = 
baseTablePartDesc.getPartSpec();
-    if(basePartSpec != null) {
-      command.append(" WHERE ");
-      List<String> pkv = getPartKVPairStringArray(basePartSpec);
-      for (int i = 0; i < pkv.size(); i++) {
-        String partKV = pkv.get(i);
-        command.append(partKV);
-        if (i < pkv.size() - 1) {
-          command.append(" AND ");
-        }
-      }
-    }
-    command.append(" GROUP BY ");
-    command.append(VirtualColumn.FILENAME.getName());
-    command.append(",");
-    command.append(VirtualColumn.BLOCKOFFSET.getName());
-    for (FieldSchema fieldSchema : indexField) {
-      command.append(",");
-      command.append(HiveUtils.unparseIdentifier(fieldSchema.getName()));
-    }
-
-    // Require clusterby ROWOFFSET if map-size aggregation is off.
-    // TODO: Make this work without map side aggregation
-    if (!builderConf.get("hive.map.aggr", null).equals("true")) {
-      throw new HiveException("Cannot construct index without map-side 
aggregation");
-    }
-
-    Task<?> rootTask = IndexUtils.createRootTask(builderConf, inputs, outputs,
-        command, partSpec, indexTableName, dbName, lineageState);
-    return rootTask;
-  }
-
-  @Override
-  /**
-   * No lower bound on bitmap index query size, so this will always return true
-   */
-  public boolean checkQuerySize(long querySize, HiveConf hiveConf) {
-    return true;
-  }
-
-  @Override
-  public boolean usesIndexTable() {
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapInnerQuery.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapInnerQuery.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapInnerQuery.java
deleted file mode 100644
index c7500a5..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapInnerQuery.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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.index.bitmap;
-
-import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.metadata.HiveUtils;
-import org.apache.hadoop.hive.ql.index.bitmap.BitmapQuery;
-
-/**
- * Representation of inner bitmap index SELECT query that scans bitmap index
- * tables for a pushed predicate
- */
-public class BitmapInnerQuery implements BitmapQuery {
-  private String tableName;
-  private ExprNodeDesc predicate;
-  private String alias;
-  private String queryStr;
-
-  public BitmapInnerQuery(String tableName, ExprNodeDesc predicate, String 
alias) {
-    this.tableName = tableName;
-    this.predicate = predicate;
-    this.alias = alias;
-    constructQueryStr();
-  }
-
-  /**
-   * Return a string representation of the query string for compilation
-   */
-  public String toString() {
-    return queryStr;
-  }
-
-  /**
-   * Construct a string representation of the query to be compiled
-   */
-  private  void constructQueryStr() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("(SELECT * FROM ");
-    sb.append(HiveUtils.unparseIdentifier(tableName));
-    sb.append(" WHERE ");
-    sb.append(predicate.getExprString());
-    sb.append(") ");
-    sb.append(alias);
-    queryStr = sb.toString();
-  }
-
-  /**
-   * Return the assigned alias of the SELECT statement
-   */
-  public String getAlias() {
-    return alias;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapObjectInput.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapObjectInput.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapObjectInput.java
deleted file mode 100644
index 0f312a3..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapObjectInput.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * 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.index.bitmap;
-
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import org.apache.hadoop.io.LongWritable;
-import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
-import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
-import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.lazy.LazyLong;
-
-/**
- * An ObjectInput that allows for conversion from an List of LongWritable
- * to an EWAH-compressed bitmap.
- */
-public class BitmapObjectInput implements ObjectInput {
-  Iterator<LongWritable> bufferIter;
-  List<LongWritable> buffer;
-
-  public BitmapObjectInput() {
-    buffer = new ArrayList<LongWritable>();
-    bufferIter = buffer.iterator();
-  }
-
-  public BitmapObjectInput(List<LongWritable> l) {
-    readFromList(l);
-  }
-
-  public void readFromList(List<LongWritable> l) {
-    buffer = l;
-    bufferIter = buffer.iterator();
-  }
-
-  @Override
-  public int available() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void close() throws IOException {
-    throw new UnsupportedOperationException();
-
-  }
-
-  @Override
-  public int read() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int read(byte[] arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int read(byte[] arg0, int arg1, int arg2) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Object readObject() throws ClassNotFoundException, IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long skip(long arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean readBoolean() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public byte readByte() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public char readChar() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public double readDouble() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public float readFloat() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void readFully(byte[] arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void readFully(byte[] arg0, int arg1, int arg2) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int readInt() throws IOException {
-    if (bufferIter.hasNext()) {
-      LongObjectInspector loi = 
PrimitiveObjectInspectorFactory.writableLongObjectInspector;
-      Long l = PrimitiveObjectInspectorUtils.getLong(bufferIter.next(), loi);
-      return l.intValue();
-      //return bufferIter.next().intValue();
-    }
-    else {
-      throw new IOException();
-    }
-  }
-
-  @Override
-  public String readLine() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long readLong() throws IOException {
-    //LongObjectInspector loi = 
PrimitiveObjectInspectorFactory.writableLongObjectInspector;
-    if (bufferIter.hasNext()) {
-      LongObjectInspector loi = 
PrimitiveObjectInspectorFactory.writableLongObjectInspector;
-      return PrimitiveObjectInspectorUtils.getLong(bufferIter.next(), loi);
-      //return bufferIter.next();
-    }
-    else {
-      throw new IOException();
-    }
-  }
-
-  @Override
-  public short readShort() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public String readUTF() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int readUnsignedByte() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int readUnsignedShort() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int skipBytes(int n) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapObjectOutput.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapObjectOutput.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapObjectOutput.java
deleted file mode 100644
index e9d959d..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapObjectOutput.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.index.bitmap;
-
-import java.io.IOException;
-import java.io.ObjectOutput;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.hadoop.io.LongWritable;
-import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
-import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
-import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-
-/**
- * An ObjectOutput that allows conversion from an EWAH-compressed bitmap
- * to an List of LongWritable.
- */
-public class BitmapObjectOutput implements ObjectOutput {
-  ArrayList<LongWritable> buffer = new ArrayList<LongWritable>();
-
-  public List<LongWritable> list() {
-    return buffer;
-  }
-
-  @Override
-  public void close() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void flush() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void write(int arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void write(byte[] arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void write(byte[] arg0, int arg1, int arg2) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeObject(Object arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeBoolean(boolean arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeByte(int arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeBytes(String arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeChar(int arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeChars(String arg0) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeDouble(double v) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeFloat(float v) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeInt(int v) throws IOException {
-    buffer.add(new LongWritable(v));
-  }
-
-  @Override
-  public void writeLong(long v) throws IOException {
-    buffer.add(new LongWritable(v));
-  }
-
-  @Override
-  public void writeShort(int v) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void writeUTF(String s) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapOuterQuery.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapOuterQuery.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapOuterQuery.java
deleted file mode 100644
index 135b1ed..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapOuterQuery.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.index.bitmap;
-
-import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.index.bitmap.BitmapQuery;
-
-/**
- * Representation of the outer query on bitmap indexes that JOINs the result of
- * inner SELECT scans on bitmap indexes (represented in BitmapQuery objects)
- * using EWAH_* bitwise operations
- */
-public class BitmapOuterQuery implements BitmapQuery {
-  private String alias;
-  private BitmapQuery lhs;
-  private BitmapQuery rhs;
-  private String queryStr;
-
-  public BitmapOuterQuery(String alias, BitmapQuery lhs, BitmapQuery rhs) {
-    this.alias = alias;
-    this.lhs = lhs;
-    this.rhs = rhs;
-    constructQueryStr();
-  }
-
-  public String getAlias() {
-    return alias;
-  }
-
-  /**
-   * Return a string representation of the query for compilation
-   */
-  public String toString() {
-    return queryStr;
-  }
-
-  /**
-   * Construct a string representation of the query to be compiled
-   */
-  private void constructQueryStr() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("(SELECT ");
-    sb.append(lhs.getAlias());
-    sb.append(".`_bucketname`, ");
-    sb.append(rhs.getAlias());
-    sb.append(".`_offset`, ");
-    sb.append("EWAH_BITMAP_AND(");
-    sb.append(lhs.getAlias());
-    sb.append(".`_bitmaps`, ");
-    sb.append(rhs.getAlias());
-    sb.append(".`_bitmaps`) AS `_bitmaps` FROM ");
-    sb.append(lhs.toString());
-    sb.append(" JOIN ");
-    sb.append(rhs.toString());
-    sb.append(" ON ");
-    sb.append(lhs.getAlias());
-    sb.append(".`_bucketname` = ");
-    sb.append(rhs.getAlias());
-    sb.append(".`_bucketname` AND ");
-    sb.append(lhs.getAlias());
-    sb.append(".`_offset` = ");
-    sb.append(rhs.getAlias());
-    sb.append(".`_offset`) ");
-    sb.append(this.alias);
-    queryStr = sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapQuery.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapQuery.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapQuery.java
deleted file mode 100644
index 4b1ff46..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/bitmap/BitmapQuery.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.index.bitmap;
-import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-
-/**
- * Generic interface to representations of queries on bitmap indexes
- */
-public interface BitmapQuery {
-  public String getAlias();
-
-  public String toString();
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/compact/CompactIndexHandler.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/compact/CompactIndexHandler.java 
b/ql/src/java/org/apache/hadoop/hive/ql/index/compact/CompactIndexHandler.java
deleted file mode 100644
index c4d02ee..0000000
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/index/compact/CompactIndexHandler.java
+++ /dev/null
@@ -1,408 +0,0 @@
-/*
- * 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.index.compact;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Set;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.common.JavaUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.Driver;
-import org.apache.hadoop.hive.ql.exec.FilterOperator;
-import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.index.HiveIndexQueryContext;
-import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer;
-import org.apache.hadoop.hive.ql.index.IndexSearchCondition;
-import org.apache.hadoop.hive.ql.index.TableBasedIndexHandler;
-import org.apache.hadoop.hive.ql.io.HiveInputFormat;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import 
org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler.DecomposedPredicate;
-import org.apache.hadoop.hive.ql.metadata.HiveUtils;
-import org.apache.hadoop.hive.ql.metadata.Partition;
-import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
-import org.apache.hadoop.hive.ql.optimizer.IndexUtils;
-import org.apache.hadoop.hive.ql.parse.ParseContext;
-import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
-import org.apache.hadoop.hive.ql.plan.MapWork;
-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.session.LineageState;
-import org.apache.hadoop.hive.ql.stats.StatsUtils;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
-
-public class CompactIndexHandler extends TableBasedIndexHandler {
-
-  // The names of the partition columns
-  private Set<String> partitionCols;
-  // Whether or not the conditions have been met to use the fact the index is 
sorted
-  private boolean useSorted;
-  private static final Logger LOG = 
LoggerFactory.getLogger(CompactIndexHandler.class.getName());
-
-
-  @Override
-  public void analyzeIndexDefinition(Table baseTable, Index index,
-      Table indexTable) throws HiveException {
-    StorageDescriptor storageDesc = index.getSd();
-    if (this.usesIndexTable() && indexTable != null) {
-      StorageDescriptor indexTableSd = storageDesc.deepCopy();
-      List<FieldSchema> indexTblCols = indexTableSd.getCols();
-      FieldSchema bucketFileName = new FieldSchema("_bucketname", "string", 
"");
-      indexTblCols.add(bucketFileName);
-      FieldSchema offSets = new FieldSchema("_offsets", "array<bigint>", "");
-      indexTblCols.add(offSets);
-      indexTable.setSd(indexTableSd);
-    }
-  }
-
-  @Override
-  protected Task<?> getIndexBuilderMapRedTask(Set<ReadEntity> inputs, 
Set<WriteEntity> outputs,
-      List<FieldSchema> indexField, boolean partitioned,
-      PartitionDesc indexTblPartDesc, String indexTableName,
-      PartitionDesc baseTablePartDesc, String baseTableName, String dbName,
-      LineageState lineageState) throws HiveException {
-
-    String indexCols = 
HiveUtils.getUnparsedColumnNamesFromFieldSchema(indexField);
-
-    //form a new insert overwrite query.
-    StringBuilder command= new StringBuilder();
-    LinkedHashMap<String, String> partSpec = indexTblPartDesc.getPartSpec();
-
-    String fullIndexTableName = 
StatsUtils.getFullyQualifiedTableName(HiveUtils.unparseIdentifier(dbName),
-        HiveUtils.unparseIdentifier(indexTableName));
-    command.append("INSERT OVERWRITE TABLE " + fullIndexTableName);
-    if (partitioned && indexTblPartDesc != null) {
-      command.append(" PARTITION ( ");
-      List<String> ret = getPartKVPairStringArray(partSpec);
-      for (int i = 0; i < ret.size(); i++) {
-        String partKV = ret.get(i);
-        command.append(partKV);
-        if (i < ret.size() - 1) {
-          command.append(",");
-        }
-      }
-      command.append(" ) ");
-    }
-
-    String fullBaseTableName = 
StatsUtils.getFullyQualifiedTableName(HiveUtils.unparseIdentifier(dbName),
-        HiveUtils.unparseIdentifier(baseTableName));
-    command.append(" SELECT ");
-    command.append(indexCols);
-    command.append(",");
-
-    command.append(VirtualColumn.FILENAME.getName());
-    command.append(",");
-    command.append(" collect_set (");
-    command.append(VirtualColumn.BLOCKOFFSET.getName());
-    command.append(") ");
-    command.append(" FROM " + fullBaseTableName);
-    LinkedHashMap<String, String> basePartSpec = 
baseTablePartDesc.getPartSpec();
-    if(basePartSpec != null) {
-      command.append(" WHERE ");
-      List<String> pkv = getPartKVPairStringArray(basePartSpec);
-      for (int i = 0; i < pkv.size(); i++) {
-        String partKV = pkv.get(i);
-        command.append(partKV);
-        if (i < pkv.size() - 1) {
-          command.append(" AND ");
-        }
-      }
-    }
-    command.append(" GROUP BY ");
-    command.append(indexCols + ", " + VirtualColumn.FILENAME.getName());
-
-    HiveConf builderConf = new HiveConf(getConf(), CompactIndexHandler.class);
-    builderConf.setBoolVar(HiveConf.ConfVars.HIVEMERGEMAPFILES, false);
-    builderConf.setBoolVar(HiveConf.ConfVars.HIVEMERGEMAPREDFILES, false);
-    builderConf.setBoolVar(HiveConf.ConfVars.HIVEMERGETEZFILES, false);
-    Task<?> rootTask = IndexUtils.createRootTask(builderConf, inputs, outputs,
-        command, partSpec, indexTableName, dbName, lineageState);
-    return rootTask;
-  }
-
-  @Override
-  public void generateIndexQuery(List<Index> indexes, ExprNodeDesc predicate,
-    ParseContext pctx, HiveIndexQueryContext queryContext) {
-
-    Index index = indexes.get(0);
-    DecomposedPredicate decomposedPredicate = decomposePredicate(predicate, 
index,
-                                                                  
queryContext.getQueryPartitions());
-
-    if (decomposedPredicate == null) {
-      queryContext.setQueryTasks(null);
-      return; // abort if we couldn't pull out anything from the predicate
-    }
-
-    // pass residual predicate back out for further processing
-    queryContext.setResidualPredicate(decomposedPredicate.residualPredicate);
-    // setup TableScanOperator to change input format for original query
-    
queryContext.setIndexInputFormat(HiveCompactIndexInputFormat.class.getName());
-
-    // Build reentrant QL for index query
-    StringBuilder qlCommand = new StringBuilder("INSERT OVERWRITE DIRECTORY ");
-
-    String tmpFile = pctx.getContext().getMRTmpPath().toUri().toString();
-    queryContext.setIndexIntermediateFile(tmpFile);
-    qlCommand.append( "\"" + tmpFile + "\" ");            // QL includes " 
around file name
-    qlCommand.append("SELECT `_bucketname` ,  `_offsets` FROM ");
-    qlCommand.append(HiveUtils.unparseIdentifier(index.getIndexTableName()));
-    qlCommand.append(" WHERE ");
-
-    String predicateString = 
decomposedPredicate.pushedPredicate.getExprString();
-    qlCommand.append(predicateString);
-
-    // generate tasks from index query string
-    LOG.info("Generating tasks for re-entrant QL query: " + 
qlCommand.toString());
-    HiveConf queryConf = new HiveConf(pctx.getConf(), 
CompactIndexHandler.class);
-    HiveConf.setBoolVar(queryConf, HiveConf.ConfVars.COMPRESSRESULT, false);
-    Driver driver = new Driver(queryConf, 
pctx.getQueryState().getLineageState());
-    driver.compile(qlCommand.toString(), false);
-
-    if (pctx.getConf().getBoolVar(ConfVars.HIVE_INDEX_COMPACT_BINARY_SEARCH) 
&& useSorted) {
-      // For now, only works if the predicate is a single condition
-      MapWork work = null;
-      String originalInputFormat = null;
-      for (Task task : driver.getPlan().getRootTasks()) {
-        // The index query should have one and only one map reduce task in the 
root tasks
-        // Otherwise something is wrong, log the problem and continue using 
the default format
-        if (task.getWork() instanceof MapredWork) {
-          if (work != null) {
-            LOG.error("Tried to use a binary search on a compact index but 
there were an " +
-                      "unexpected number (>1) of root level map reduce tasks 
in the " +
-                      "reentrant query plan.");
-            work.setInputformat(null);
-            work.setInputFormatSorted(false);
-            break;
-          }
-          if (task.getWork() != null) {
-            work = ((MapredWork)task.getWork()).getMapWork();
-          }
-          String inputFormat = work.getInputformat();
-          originalInputFormat = inputFormat;
-          if (inputFormat == null) {
-            inputFormat = HiveConf.getVar(pctx.getConf(), 
HiveConf.ConfVars.HIVEINPUTFORMAT);
-          }
-
-          // We can only perform a binary search with HiveInputFormat and 
CombineHiveInputFormat
-          // and BucketizedHiveInputFormat
-          try {
-            if 
(!HiveInputFormat.class.isAssignableFrom(JavaUtils.loadClass(inputFormat))) {
-              work = null;
-              break;
-            }
-          } catch (ClassNotFoundException e) {
-            LOG.error("Map reduce work's input format class: " + inputFormat + 
" was not found. " +
-                       "Cannot use the fact the compact index is sorted.");
-            work = null;
-            break;
-          }
-
-          work.setInputFormatSorted(true);
-        }
-      }
-
-      if (work != null) {
-        // Find the filter operator and expr node which act on the index 
column and mark them
-        if (!findIndexColumnFilter(work.getAliasToWork().values())) {
-          LOG.error("Could not locate the index column's filter operator and 
expr node. Cannot " +
-                    "use the fact the compact index is sorted.");
-          work.setInputformat(originalInputFormat);
-          work.setInputFormatSorted(false);
-        }
-      }
-    }
-
-
-    queryContext.addAdditionalSemanticInputs(driver.getPlan().getInputs());
-    queryContext.setQueryTasks(driver.getPlan().getRootTasks());
-    return;
-  }
-
-  /**
-   * Does a depth first search on the operator tree looking for a filter 
operator whose predicate
-   * has one child which is a column which is not in the partition
-   * @param operators
-   * @return whether or not it has found its target
-   */
-  private boolean findIndexColumnFilter(
-    Collection<Operator<? extends OperatorDesc>> operators) {
-    for (Operator<? extends OperatorDesc> op : operators) {
-      if (op instanceof FilterOperator &&
-        ((FilterOperator)op).getConf().getPredicate().getChildren() != null) {
-        // Is this the target
-        if 
(findIndexColumnExprNodeDesc(((FilterOperator)op).getConf().getPredicate())) {
-          ((FilterOperator)op).getConf().setSortedFilter(true);
-          return true;
-        }
-      }
-
-      // If the target has been found, no need to continue
-      if (findIndexColumnFilter(op.getChildOperators())) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private boolean findIndexColumnExprNodeDesc(ExprNodeDesc expression) {
-    if (expression.getChildren() == null) {
-      return false;
-    }
-
-    if (expression.getChildren().size() == 2) {
-      ExprNodeColumnDesc columnDesc = null;
-      if (expression.getChildren().get(0) instanceof ExprNodeColumnDesc) {
-        columnDesc = (ExprNodeColumnDesc)expression.getChildren().get(0);
-      } else if (expression.getChildren().get(1) instanceof 
ExprNodeColumnDesc) {
-        columnDesc = (ExprNodeColumnDesc)expression.getChildren().get(1);
-      }
-
-      // Is this the target
-      if (columnDesc != null && 
!partitionCols.contains(columnDesc.getColumn())) {
-        assert expression instanceof ExprNodeGenericFuncDesc :
-               "Expression containing index column is does not support 
sorting, should not try" +
-               "and sort";
-        ((ExprNodeGenericFuncDesc)expression).setSortedExpr(true);
-        return true;
-      }
-    }
-
-    for (ExprNodeDesc child : expression.getChildren()) {
-      // If the target has been found, no need to continue
-      if (findIndexColumnExprNodeDesc(child)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Split the predicate into the piece we can deal with (pushed), and the one 
we can't (residual)
-   * @param predicate
-   * @param index
-   * @return
-   */
-  private DecomposedPredicate decomposePredicate(ExprNodeDesc predicate, Index 
index,
-      Set<Partition> queryPartitions) {
-    IndexPredicateAnalyzer analyzer = getIndexPredicateAnalyzer(index, 
queryPartitions);
-    List<IndexSearchCondition> searchConditions = new 
ArrayList<IndexSearchCondition>();
-    // split predicate into pushed (what we can handle), and residual (what we 
can't handle)
-    ExprNodeGenericFuncDesc residualPredicate = 
(ExprNodeGenericFuncDesc)analyzer.
-      analyzePredicate(predicate, searchConditions);
-
-    if (searchConditions.size() == 0) {
-      return null;
-    }
-
-    int numIndexCols = 0;
-    for (IndexSearchCondition searchCondition : searchConditions) {
-      if 
(!partitionCols.contains(searchCondition.getColumnDesc().getColumn())) {
-        numIndexCols++;
-      }
-    }
-
-    // For now, only works if the predicate has a single condition on an index 
column
-    if (numIndexCols == 1) {
-      useSorted = true;
-    } else {
-      useSorted = false;
-    }
-
-    DecomposedPredicate decomposedPredicate = new DecomposedPredicate();
-    decomposedPredicate.pushedPredicate = 
analyzer.translateSearchConditions(searchConditions);
-    decomposedPredicate.residualPredicate = residualPredicate;
-
-    return decomposedPredicate;
-  }
-
-  /**
-   * Instantiate a new predicate analyzer suitable for determining
-   * whether we can use an index, based on rules for indexes in
-   * WHERE clauses that we support
-   *
-   * @return preconfigured predicate analyzer for WHERE queries
-   */
-  private IndexPredicateAnalyzer getIndexPredicateAnalyzer(Index index, 
Set<Partition> queryPartitions)  {
-    IndexPredicateAnalyzer analyzer = new IndexPredicateAnalyzer();
-
-    analyzer.addComparisonOp(GenericUDFOPEqual.class.getName());
-    analyzer.addComparisonOp(GenericUDFOPLessThan.class.getName());
-    analyzer.addComparisonOp(GenericUDFOPEqualOrLessThan.class.getName());
-    analyzer.addComparisonOp(GenericUDFOPGreaterThan.class.getName());
-    analyzer.addComparisonOp(GenericUDFOPEqualOrGreaterThan.class.getName());
-
-    // only return results for columns in this index
-    List<FieldSchema> columnSchemas = index.getSd().getCols();
-    for (FieldSchema column : columnSchemas) {
-      analyzer.allowColumnName(column.getName());
-    }
-
-    // partitioned columns are treated as if they have indexes so that the 
partitions
-    // are used during the index query generation
-    partitionCols = new HashSet<String>();
-    for (Partition part : queryPartitions) {
-      if (part.getSpec().isEmpty()) {
-        continue; // empty partitions are from whole tables, so we don't want 
to add them in
-      }
-      for (String column : part.getSpec().keySet()) {
-        analyzer.allowColumnName(column);
-        partitionCols.add(column);
-      }
-    }
-
-    return analyzer;
-  }
-
-
-  @Override
-  public boolean checkQuerySize(long querySize, HiveConf hiveConf) {
-    long minSize = 
hiveConf.getLongVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER_COMPACT_MINSIZE);
-    long maxSize = 
hiveConf.getLongVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER_COMPACT_MAXSIZE);
-    if (maxSize < 0) {
-      maxSize = Long.MAX_VALUE;
-    }
-    return (querySize > minSize & querySize < maxSize);
-  }
-
-  @Override
-  public boolean usesIndexTable() {
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/index/compact/HiveCompactIndexInputFormat.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/index/compact/HiveCompactIndexInputFormat.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/index/compact/HiveCompactIndexInputFormat.java
deleted file mode 100644
index 6d9c968..0000000
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/index/compact/HiveCompactIndexInputFormat.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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.index.compact;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat;
-
-public class HiveCompactIndexInputFormat extends HiveIndexedInputFormat {
-
-  public static final Logger l4j =
-    LoggerFactory.getLogger(HiveCompactIndexInputFormat.class.getSimpleName());
-
-  public HiveCompactIndexInputFormat() {
-    super("hive.index.compact.file");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index 5cd30cb..912eb10 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -23,6 +23,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -41,6 +42,7 @@ import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hive.common.util.Ref;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -105,6 +107,22 @@ public class HiveInputFormat<K extends WritableComparable, 
V extends Writable>
   protected Map<Path, PartitionDesc> pathToPartitionInfo;
   protected MapWork mrwork;
 
+  public static final class HiveInputSplitComparator implements 
Comparator<HiveInputSplit> {
+    @Override
+    public int compare(HiveInputSplit o1, HiveInputSplit o2) {
+      int pathCompare = comparePath(o1.getPath(), o2.getPath());
+      if (pathCompare != 0) {
+        return pathCompare;
+      }
+      return Long.compare(o1.getStart(), o2.getStart());
+    }
+
+    private int comparePath(Path p1, Path p2) {
+      return p1.compareTo(p2);
+    }
+  }
+
+
   /**
    * HiveInputSplit encapsulates an InputSplit with its corresponding
    * inputFormatClass. The reason that it derives from FileSplit is to make 
sure
@@ -113,6 +131,7 @@ public class HiveInputFormat<K extends WritableComparable, 
V extends Writable>
   public static class HiveInputSplit extends FileSplit implements InputSplit,
       Configurable {
 
+
     InputSplit inputSplit;
     String inputFormatClassName;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b0d3cb45/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java
index 2930a46..bb75ebf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java
@@ -161,7 +161,9 @@ public class ExternalCache implements FooterCache {
 
   private boolean processBbResult(
       ByteBuffer bb, int ix, HdfsFileStatusWithId file, OrcTail[] result) 
throws IOException {
-    if (bb == null) return true;
+    if (bb == null) {
+      return true;
+    }
     result[ix] = createOrcTailFromMs(file, bb);
     if (result[ix] == null) {
       return false;
@@ -173,7 +175,10 @@ public class ExternalCache implements FooterCache {
 
   private void processPpdResult(MetadataPpdResult mpr, HdfsFileStatusWithId 
file,
       int ix, OrcTail[] result, ByteBuffer[] ppdResult) throws IOException {
-    if (mpr == null) return; // This file is unknown to metastore.
+    if (mpr == null)
+     {
+      return; // This file is unknown to metastore.
+    }
 
     ppdResult[ix] = mpr.isSetIncludeBitset() ? mpr.bufferForIncludeBitset() : 
NO_SPLIT_AFTER_PPD;
     if (mpr.isSetMetadata()) {
@@ -187,7 +192,9 @@ public class ExternalCache implements FooterCache {
   private List<Long> determineFileIdsToQuery(
       List<HdfsFileStatusWithId> files, OrcTail[] result, HashMap<Long, 
Integer> posMap) {
     for (int i = 0; i < result.length; ++i) {
-      if (result[i] != null) continue;
+      if (result[i] != null) {
+        continue;
+      }
       HdfsFileStatusWithId file = files.get(i);
       final FileStatus fs = file.getFileStatus();
       Long fileId = file.getFileId();
@@ -224,9 +231,13 @@ public class ExternalCache implements FooterCache {
   }
 
   private ByteBuffer getSerializedSargForMetastore(boolean isOriginal) {
-    if (sarg == null) return null;
+    if (sarg == null) {
+      return null;
+    }
     ByteBuffer serializedSarg = isOriginal ? sargIsOriginal : 
sargNotIsOriginal;
-    if (serializedSarg != null) return serializedSarg;
+    if (serializedSarg != null) {
+      return serializedSarg;
+    }
     SearchArgument sarg2 = sarg;
     Kryo kryo = SerializationUtilities.borrowKryo();
     try {
@@ -292,7 +303,9 @@ public class ExternalCache implements FooterCache {
 
   private static OrcTail createOrcTailFromMs(
       HdfsFileStatusWithId file, ByteBuffer bb) throws IOException {
-    if (bb == null) return null;
+    if (bb == null) {
+      return null;
+    }
     FileStatus fs = file.getFileStatus();
     ByteBuffer copy = bb.duplicate();
     try {

Reply via email to