You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/12/21 21:13:21 UTC

[1/2] hive git commit: HIVE-12632 : LLAP: don't use IO elevator for ACID tables (Sergey Shelukhin, reviewed by Prasanth Jayachandran and Alan Gates)

Repository: hive
Updated Branches:
  refs/heads/branch-2.0 2b16a1ee3 -> a53d8af4a
  refs/heads/master ac6ba5031 -> 13aa15d11


HIVE-12632 : LLAP: don't use IO elevator for ACID tables (Sergey Shelukhin, reviewed by Prasanth Jayachandran and Alan Gates)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/13aa15d1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/13aa15d1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/13aa15d1

Branch: refs/heads/master
Commit: 13aa15d112c152cf5cde53590a0bf4e5b98d2e6a
Parents: ac6ba50
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Dec 21 12:02:41 2015 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Dec 21 12:02:41 2015 -0800

----------------------------------------------------------------------
 .../hive/llap/io/api/impl/LlapInputFormat.java  |  39 ++-
 .../llap/io/decode/ColumnVectorProducer.java    |   4 +-
 .../llap/io/decode/OrcColumnVectorProducer.java |   4 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |   4 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |  19 ++
 .../hadoop/hive/ql/io/LlapAwareSplit.java       |  27 ++
 .../apache/hadoop/hive/ql/io/orc/OrcSplit.java  |  19 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   7 +-
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |  36 ++-
 ql/src/test/queries/clientpositive/llap_acid.q  |  53 ++++
 .../test/results/clientpositive/llap_acid.q.out | 307 +++++++++++++++++++
 11 files changed, 485 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
index b57366c..a3d71c0 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.llap.ConsumerFeedback;
 import org.apache.hadoop.hive.llap.DebugUtils;
 import org.apache.hadoop.hive.llap.counters.QueryFragmentCounters;
@@ -33,12 +35,14 @@ import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat.AvoidSplitCombination;
+import org.apache.hadoop.hive.ql.io.LlapAwareSplit;
 import org.apache.hadoop.hive.ql.io.SelfDescribingInputFormatInterface;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer;
 import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.io.NullWritable;
@@ -55,11 +59,12 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 
-public class LlapInputFormat
-  implements InputFormat<NullWritable, VectorizedRowBatch>, VectorizedInputFormatInterface,
-  SelfDescribingInputFormatInterface {
+public class LlapInputFormat implements InputFormat<NullWritable, VectorizedRowBatch>,
+    VectorizedInputFormatInterface, SelfDescribingInputFormatInterface,
+    AvoidSplitCombination {
   @SuppressWarnings("rawtypes")
   private final InputFormat sourceInputFormat;
+  private final AvoidSplitCombination sourceASC;
   private final ColumnVectorProducer cvp;
   private final ListeningExecutorService executor;
   private final String hostName;
@@ -73,16 +78,29 @@ public class LlapInputFormat
     this.executor = executor;
     this.cvp = cvp;
     this.sourceInputFormat = sourceInputFormat;
+    this.sourceASC = (sourceInputFormat instanceof AvoidSplitCombination)
+        ? (AvoidSplitCombination)sourceInputFormat : null;
     this.hostName = HiveStringUtils.getHostname();
   }
 
   @Override
   public RecordReader<NullWritable, VectorizedRowBatch> getRecordReader(
       InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    boolean useLlapIo = true;
+    if (split instanceof LlapAwareSplit) {
+      useLlapIo = ((LlapAwareSplit)split).canUseLlapIo();
+    }
+    if (!useLlapIo) {
+      LlapIoImpl.LOG.warn("Not using LLAP IO for an unsupported split: " + split);
+      @SuppressWarnings("unchecked")
+      RecordReader<NullWritable, VectorizedRowBatch> rr =
+          sourceInputFormat.getRecordReader(split, job, reporter);
+      return rr;
+    }
     boolean isVectorMode = Utilities.isVectorMode(job);
     if (!isVectorMode) {
-      LlapIoImpl.LOG.error("No llap in non-vectorized mode");
-      throw new UnsupportedOperationException("No llap in non-vectorized mode");
+      LlapIoImpl.LOG.error("No LLAP IO in non-vectorized mode");
+      throw new UnsupportedOperationException("No LLAP IO in non-vectorized mode");
     }
     FileSplit fileSplit = (FileSplit)split;
     reporter.setStatus(fileSplit.toString());
@@ -102,7 +120,7 @@ public class LlapInputFormat
 
   private class LlapRecordReader
       implements RecordReader<NullWritable, VectorizedRowBatch>, Consumer<ColumnVectorBatch> {
-    private final InputSplit split;
+    private final FileSplit split;
     private final List<Integer> columnIds;
     private final SearchArgument sarg;
     private final String[] columnNames;
@@ -140,7 +158,7 @@ public class LlapInputFormat
       int partitionColumnCount = rbCtx.getPartitionColumnCount();
       if (partitionColumnCount > 0) {
         partitionValues = new Object[partitionColumnCount];
-        rbCtx.getPartitionValues(rbCtx, job, split, partitionValues);
+        VectorizedRowBatchCtx.getPartitionValues(rbCtx, job, split, partitionValues);
       } else {
         partitionValues = null;
       }
@@ -328,4 +346,9 @@ public class LlapInputFormat
       return 0.0f;
     }
   }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return sourceASC == null ? false : sourceASC.shouldSkipCombine(path, conf);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java
index 5c34fa3..b3b571d 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java
@@ -24,13 +24,13 @@ import org.apache.hadoop.hive.llap.counters.QueryFragmentCounters;
 import org.apache.hadoop.hive.llap.io.api.impl.ColumnVectorBatch;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.FileSplit;
 
 /**
  * Entry point used by LlapInputFormat to create read pipeline to get data.
  */
 public interface ColumnVectorProducer {
-  ReadPipeline createReadPipeline(Consumer<ColumnVectorBatch> consumer, InputSplit split,
+  ReadPipeline createReadPipeline(Consumer<ColumnVectorBatch> consumer, FileSplit split,
       List<Integer> columnIds, SearchArgument sarg, String[] columnNames,
       QueryFragmentCounters counters);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
index 259c483..e156eaa 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hive.llap.metrics.LlapDaemonQueueMetrics;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer;
 import org.apache.hadoop.hive.ql.io.orc.encoded.OrcCacheKey;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.FileSplit;
 
 public class OrcColumnVectorProducer implements ColumnVectorProducer {
 
@@ -65,7 +65,7 @@ public class OrcColumnVectorProducer implements ColumnVectorProducer {
 
   @Override
   public ReadPipeline createReadPipeline(
-      Consumer<ColumnVectorBatch> consumer, InputSplit split,
+      Consumer<ColumnVectorBatch> consumer, FileSplit split,
       List<Integer> columnIds, SearchArgument sarg, String[] columnNames,
       QueryFragmentCounters counters) {
     cacheMetrics.incrCacheReadRequests();

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
index 5957d57..58d2ac8 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
@@ -161,14 +161,14 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
   private volatile boolean isPaused = false;
 
   public OrcEncodedDataReader(LowLevelCache lowLevelCache, Cache<OrcCacheKey> cache,
-      OrcMetadataCache metadataCache, Configuration conf, InputSplit split,
+      OrcMetadataCache metadataCache, Configuration conf, FileSplit split,
       List<Integer> columnIds, SearchArgument sarg, String[] columnNames,
       OrcEncodedDataConsumer consumer, QueryFragmentCounters counters) {
     this.lowLevelCache = lowLevelCache;
     this.metadataCache = metadataCache;
     this.cache = cache;
     this.conf = conf;
-    this.split = (FileSplit)split;
+    this.split = split;
     this.columnIds = columnIds;
     if (this.columnIds != null) {
       Collections.sort(this.columnIds);

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index 24137e2..f248054 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.shims.HadoopShims.HdfsFileStatusWithId;
@@ -36,6 +37,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.regex.Pattern;
 
 /**
@@ -655,4 +658,20 @@ public class AcidUtils {
       }
     }
   }
+
+  public static boolean isTablePropertyTransactional(Properties props) {
+    String resultStr = props.getProperty(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL);
+    if (resultStr == null) {
+      resultStr = props.getProperty(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.toUpperCase());
+    }
+    return resultStr != null && resultStr.equalsIgnoreCase("true");
+  }
+
+  public static boolean isTablePropertyTransactional(Map<String, String> parameters) {
+    String resultStr = parameters.get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL);
+    if (resultStr == null) {
+      resultStr = parameters.get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.toUpperCase());
+    }
+    return resultStr != null && resultStr.equalsIgnoreCase("true");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/ql/src/java/org/apache/hadoop/hive/ql/io/LlapAwareSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/LlapAwareSplit.java b/ql/src/java/org/apache/hadoop/hive/ql/io/LlapAwareSplit.java
new file mode 100644
index 0000000..ead4678
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/LlapAwareSplit.java
@@ -0,0 +1,27 @@
+/**
+ * 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.io;
+
+/**
+ * Split that is aware that it could be executed in LLAP. Allows LlapInputFormat to do
+ * a last-minute check to see of LLAP IO pipeline should be used for this particular split.
+ * By default, there is no such check - whatever is sent in is attempted with LLAP IO.
+ */
+public interface LlapAwareSplit {
+  boolean canUseLlapIo();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
index 61cde41..76f1328 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
@@ -31,6 +31,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.io.ColumnarSplit;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
+import org.apache.hadoop.hive.ql.io.LlapAwareSplit;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.mapred.FileSplit;
@@ -41,9 +42,7 @@ import org.apache.hadoop.mapred.FileSplit;
  * OrcFileSplit. Holds file meta info
  *
  */
-public class OrcSplit extends FileSplit implements ColumnarSplit {
-  private static final Logger LOG = LoggerFactory.getLogger(OrcSplit.class);
-
+public class OrcSplit extends FileSplit implements ColumnarSplit, LlapAwareSplit {
   private FileMetaInfo fileMetaInfo;
   private boolean hasFooter;
   private boolean isOriginal;
@@ -58,7 +57,7 @@ public class OrcSplit extends FileSplit implements ColumnarSplit {
   static final int ORIGINAL_FLAG = 2;
   static final int FOOTER_FLAG = 1;
 
-  protected OrcSplit(){
+  protected OrcSplit() {
     //The FileSplit() constructor in hadoop 0.20 and 1.x is package private so can't use it.
     //This constructor is used to create the object and then call readFields()
     // so just pass nulls to this super constructor.
@@ -186,4 +185,16 @@ public class OrcSplit extends FileSplit implements ColumnarSplit {
   public long getColumnarProjectionSize() {
     return projColsUncompressedSize;
   }
+
+  @Override
+  public boolean canUseLlapIo() {
+    return isOriginal && (deltas == null || deltas.isEmpty());
+  }
+
+  @Override
+  public String toString() {
+    return "OrcSplit [" + getPath() + ", start=" + getStart() + ", length=" + getLength()
+        + ", isOriginal=" + isOriginal + ", hasBase=" + hasBase + ", deltas="
+        + (deltas == null ? 0 : deltas.size()) + "]";
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index ea776ca..403eda2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -12188,12 +12188,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   public static boolean isAcidTable(Table tab) {
     if (tab == null) return false;
     if (!SessionState.get().getTxnMgr().supportsAcid()) return false;
-    String tableIsTransactional =
-        tab.getProperty(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL);
-    if(tableIsTransactional == null) {
-      tableIsTransactional = tab.getProperty(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.toUpperCase());
-    }
-    return tableIsTransactional != null && tableIsTransactional.equalsIgnoreCase("true");
+    return AcidUtils.isTablePropertyTransactional(tab.getParameters());
   }
 
   private boolean isAcidOutputFormat(Class<? extends OutputFormat> of) {

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
index edafdb1..e1abf37 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorUtils;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol;
 import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.SortCol;
@@ -59,7 +60,7 @@ import com.google.common.collect.Interner;
  * This class is also used in the explain command any property with the
  * appropriate annotation will be displayed in the explain output.
  */
-@SuppressWarnings({"serial", "deprecation"})
+@SuppressWarnings({"serial"})
 public class MapWork extends BaseWork {
 
   // use LinkedHashMap to make sure the iteration order is
@@ -205,22 +206,37 @@ public class MapWork extends BaseWork {
   }
 
   public void deriveLlap(Configuration conf) {
-    boolean hasLlap = false, hasNonLlap = false;
+    boolean hasLlap = false, hasNonLlap = false, hasAcid = false;
     boolean isLlapOn = HiveInputFormat.isLlapEnabled(conf),
         canWrapAny = isLlapOn && HiveInputFormat.canWrapAnyForLlap(conf, this);
     boolean hasPathToPartInfo = (pathToPartitionInfo != null && !pathToPartitionInfo.isEmpty());
     if (canWrapAny && hasPathToPartInfo) {
+      assert isLlapOn;
       for (PartitionDesc part : pathToPartitionInfo.values()) {
-        boolean isUsingLlapIo = isLlapOn
-            && HiveInputFormat.canWrapForLlap(part.getInputFileFormatClass());
-        hasLlap |= isUsingLlapIo;
-        hasNonLlap |= (!isUsingLlapIo);
+        boolean isUsingLlapIo = HiveInputFormat.canWrapForLlap(part.getInputFileFormatClass());
+        if (isUsingLlapIo) {
+          if (part.getTableDesc() != null &&
+              AcidUtils.isTablePropertyTransactional(part.getTableDesc().getProperties())) {
+            hasAcid = true;
+          } else {
+            hasLlap = true;
+          }
+        } else {
+          hasNonLlap = true;
+        }
       }
-    } else {
-      hasNonLlap = true;
     }
-    llapIoDesc = isLlapOn ? (canWrapAny ? (hasPathToPartInfo ? ((hasLlap == hasNonLlap) ?
-        "some inputs" : (hasLlap ? "all inputs" : "no inputs")) : "unknown") : "no inputs") : null;
+    llapIoDesc = deriveLlapIoDescString(
+        isLlapOn, canWrapAny, hasPathToPartInfo, hasLlap, hasNonLlap, hasAcid);
+  }
+
+  private static String deriveLlapIoDescString(boolean isLlapOn, boolean canWrapAny,
+      boolean hasPathToPartInfo, boolean hasLlap, boolean hasNonLlap, boolean hasAcid) {
+    if (!isLlapOn) return null; // LLAP IO is off, don't output.
+    if (!canWrapAny) return "no inputs"; // Cannot use with input formats.
+    if (!hasPathToPartInfo) return "unknown"; // No information to judge.
+    if (hasAcid) return "may be used (ACID table)";
+    return (hasLlap ? (hasNonLlap ? "some inputs" : "all inputs") : "no inputs");
   }
 
   public void internTable(Interner<TableDesc> interner) {

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/ql/src/test/queries/clientpositive/llap_acid.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/llap_acid.q b/ql/src/test/queries/clientpositive/llap_acid.q
new file mode 100644
index 0000000..6bd216a
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/llap_acid.q
@@ -0,0 +1,53 @@
+set hive.mapred.mode=nonstrict;
+SET hive.vectorized.execution.enabled=true;
+
+SET hive.llap.io.enabled=false;
+
+SET hive.exec.orc.default.buffer.size=32768;
+SET hive.exec.orc.default.row.index.stride=1000;
+SET hive.optimize.index.filter=true;
+set hive.fetch.task.conversion=none;
+
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+DROP TABLE orc_llap;
+
+CREATE TABLE orc_llap (
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE)
+partitioned by (csmallint smallint)
+clustered by (cint) into 2 buckets stored as orc;
+
+insert into table orc_llap partition (csmallint = 1)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10;
+insert into table orc_llap partition (csmallint = 2)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10;
+
+alter table orc_llap SET TBLPROPERTIES ('transactional'='true');
+
+insert into table orc_llap partition (csmallint = 3)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble desc limit 10;
+
+SET hive.llap.io.enabled=true;
+
+explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint;
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint;
+
+insert into table orc_llap partition (csmallint = 1) values (1, 1, 1, 1);
+
+update orc_llap set cbigint = 2 where cint = 1;
+
+explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint;
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint;
+
+DROP TABLE orc_llap;

http://git-wip-us.apache.org/repos/asf/hive/blob/13aa15d1/ql/src/test/results/clientpositive/llap_acid.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap_acid.q.out b/ql/src/test/results/clientpositive/llap_acid.q.out
new file mode 100644
index 0000000..4ad611d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap_acid.q.out
@@ -0,0 +1,307 @@
+PREHOOK: query: DROP TABLE orc_llap
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE orc_llap
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE orc_llap (
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE)
+partitioned by (csmallint smallint)
+clustered by (cint) into 2 buckets stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@orc_llap
+POSTHOOK: query: CREATE TABLE orc_llap (
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE)
+partitioned by (csmallint smallint)
+clustered by (cint) into 2 buckets stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@orc_llap
+PREHOOK: query: insert into table orc_llap partition (csmallint = 1)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: query: insert into table orc_llap partition (csmallint = 1)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cbigint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cdouble SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cfloat SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+PREHOOK: query: insert into table orc_llap partition (csmallint = 2)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@orc_llap@csmallint=2
+POSTHOOK: query: insert into table orc_llap partition (csmallint = 2)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@orc_llap@csmallint=2
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=2).cbigint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=2).cdouble SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=2).cfloat SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=2).cint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+PREHOOK: query: alter table orc_llap SET TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: ALTERTABLE_PROPERTIES
+PREHOOK: Input: default@orc_llap
+PREHOOK: Output: default@orc_llap
+POSTHOOK: query: alter table orc_llap SET TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: ALTERTABLE_PROPERTIES
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Output: default@orc_llap
+PREHOOK: query: insert into table orc_llap partition (csmallint = 3)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble desc limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@orc_llap@csmallint=3
+POSTHOOK: query: insert into table orc_llap partition (csmallint = 3)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble desc limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@orc_llap@csmallint=3
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=3).cbigint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=3).cdouble SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=3).cfloat SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=3).cint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+PREHOOK: query: explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: orc_llap
+            filterExpr: cint is not null (type: boolean)
+            Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: cint is not null (type: boolean)
+              Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: cint (type: int), csmallint (type: smallint), cbigint (type: bigint)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: int)
+                  sort order: ++
+                  Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col2 (type: bigint)
+      Execution mode: vectorized
+      LLAP IO: may be used (ACID table)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey1 (type: int), KEY.reducesinkkey0 (type: smallint), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_llap
+PREHOOK: Input: default@orc_llap@csmallint=1
+PREHOOK: Input: default@orc_llap@csmallint=2
+PREHOOK: Input: default@orc_llap@csmallint=3
+#### A masked pattern was here ####
+POSTHOOK: query: select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Input: default@orc_llap@csmallint=1
+POSTHOOK: Input: default@orc_llap@csmallint=2
+POSTHOOK: Input: default@orc_llap@csmallint=3
+#### A masked pattern was here ####
+-285355633	1	-1241163445
+-109813638	1	-58941842
+164554497	1	1161977292
+199879534	1	123351087
+246423894	1	-1645852809
+354670578	1	562841852
+455419170	1	1108177470
+665801232	1	480783141
+708885482	1	-1645852809
+-285355633	2	-1241163445
+-109813638	2	-58941842
+164554497	2	1161977292
+199879534	2	123351087
+246423894	2	-1645852809
+354670578	2	562841852
+455419170	2	1108177470
+665801232	2	480783141
+708885482	2	-1645852809
+-923308739	3	-1887561756
+-3728	3	-1887561756
+762	3	-1645852809
+6981	3	-1887561756
+253665376	3	NULL
+497728223	3	-1887561756
+528534767	3	NULL
+528534767	3	NULL
+528534767	3	NULL
+528534767	3	NULL
+PREHOOK: query: insert into table orc_llap partition (csmallint = 1) values (1, 1, 1, 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: query: insert into table orc_llap partition (csmallint = 1) values (1, 1, 1, 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cbigint EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cdouble EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cfloat EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cint EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: update orc_llap set cbigint = 2 where cint = 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_llap
+PREHOOK: Input: default@orc_llap@csmallint=1
+PREHOOK: Input: default@orc_llap@csmallint=2
+PREHOOK: Input: default@orc_llap@csmallint=3
+PREHOOK: Output: default@orc_llap@csmallint=1
+PREHOOK: Output: default@orc_llap@csmallint=2
+PREHOOK: Output: default@orc_llap@csmallint=3
+POSTHOOK: query: update orc_llap set cbigint = 2 where cint = 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Input: default@orc_llap@csmallint=1
+POSTHOOK: Input: default@orc_llap@csmallint=2
+POSTHOOK: Input: default@orc_llap@csmallint=3
+POSTHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: Output: default@orc_llap@csmallint=2
+POSTHOOK: Output: default@orc_llap@csmallint=3
+PREHOOK: query: explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: orc_llap
+            filterExpr: cint is not null (type: boolean)
+            Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: cint is not null (type: boolean)
+              Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: cint (type: int), csmallint (type: smallint), cbigint (type: bigint)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: int)
+                  sort order: ++
+                  Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col2 (type: bigint)
+      Execution mode: vectorized
+      LLAP IO: may be used (ACID table)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey1 (type: int), KEY.reducesinkkey0 (type: smallint), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_llap
+PREHOOK: Input: default@orc_llap@csmallint=1
+PREHOOK: Input: default@orc_llap@csmallint=2
+PREHOOK: Input: default@orc_llap@csmallint=3
+#### A masked pattern was here ####
+POSTHOOK: query: select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Input: default@orc_llap@csmallint=1
+POSTHOOK: Input: default@orc_llap@csmallint=2
+POSTHOOK: Input: default@orc_llap@csmallint=3
+#### A masked pattern was here ####
+-285355633	1	-1241163445
+-109813638	1	-58941842
+1	1	2
+164554497	1	1161977292
+199879534	1	123351087
+246423894	1	-1645852809
+354670578	1	562841852
+455419170	1	1108177470
+665801232	1	480783141
+708885482	1	-1645852809
+-285355633	2	-1241163445
+-109813638	2	-58941842
+164554497	2	1161977292
+199879534	2	123351087
+246423894	2	-1645852809
+354670578	2	562841852
+455419170	2	1108177470
+665801232	2	480783141
+708885482	2	-1645852809
+-923308739	3	-1887561756
+-3728	3	-1887561756
+762	3	-1645852809
+6981	3	-1887561756
+253665376	3	NULL
+497728223	3	-1887561756
+528534767	3	NULL
+528534767	3	NULL
+528534767	3	NULL
+528534767	3	NULL
+PREHOOK: query: DROP TABLE orc_llap
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@orc_llap
+PREHOOK: Output: default@orc_llap
+POSTHOOK: query: DROP TABLE orc_llap
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Output: default@orc_llap


[2/2] hive git commit: HIVE-12632 : LLAP: don't use IO elevator for ACID tables (Sergey Shelukhin, reviewed by Prasanth Jayachandran and Alan Gates)

Posted by se...@apache.org.
HIVE-12632 : LLAP: don't use IO elevator for ACID tables (Sergey Shelukhin, reviewed by Prasanth Jayachandran and Alan Gates)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a53d8af4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a53d8af4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a53d8af4

Branch: refs/heads/branch-2.0
Commit: a53d8af4a5d61d4d2788b4b696ececd2220c1f57
Parents: 2b16a1e
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Dec 21 12:02:41 2015 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Dec 21 12:10:14 2015 -0800

----------------------------------------------------------------------
 .../hive/llap/io/api/impl/LlapInputFormat.java  |  39 ++-
 .../llap/io/decode/ColumnVectorProducer.java    |   4 +-
 .../llap/io/decode/OrcColumnVectorProducer.java |   4 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |   4 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |  19 ++
 .../hadoop/hive/ql/io/LlapAwareSplit.java       |  27 ++
 .../apache/hadoop/hive/ql/io/orc/OrcSplit.java  |  19 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   7 +-
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |  36 ++-
 ql/src/test/queries/clientpositive/llap_acid.q  |  53 ++++
 .../test/results/clientpositive/llap_acid.q.out | 307 +++++++++++++++++++
 11 files changed, 485 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
index b57366c..a3d71c0 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.llap.ConsumerFeedback;
 import org.apache.hadoop.hive.llap.DebugUtils;
 import org.apache.hadoop.hive.llap.counters.QueryFragmentCounters;
@@ -33,12 +35,14 @@ import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat.AvoidSplitCombination;
+import org.apache.hadoop.hive.ql.io.LlapAwareSplit;
 import org.apache.hadoop.hive.ql.io.SelfDescribingInputFormatInterface;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer;
 import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.io.NullWritable;
@@ -55,11 +59,12 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 
-public class LlapInputFormat
-  implements InputFormat<NullWritable, VectorizedRowBatch>, VectorizedInputFormatInterface,
-  SelfDescribingInputFormatInterface {
+public class LlapInputFormat implements InputFormat<NullWritable, VectorizedRowBatch>,
+    VectorizedInputFormatInterface, SelfDescribingInputFormatInterface,
+    AvoidSplitCombination {
   @SuppressWarnings("rawtypes")
   private final InputFormat sourceInputFormat;
+  private final AvoidSplitCombination sourceASC;
   private final ColumnVectorProducer cvp;
   private final ListeningExecutorService executor;
   private final String hostName;
@@ -73,16 +78,29 @@ public class LlapInputFormat
     this.executor = executor;
     this.cvp = cvp;
     this.sourceInputFormat = sourceInputFormat;
+    this.sourceASC = (sourceInputFormat instanceof AvoidSplitCombination)
+        ? (AvoidSplitCombination)sourceInputFormat : null;
     this.hostName = HiveStringUtils.getHostname();
   }
 
   @Override
   public RecordReader<NullWritable, VectorizedRowBatch> getRecordReader(
       InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    boolean useLlapIo = true;
+    if (split instanceof LlapAwareSplit) {
+      useLlapIo = ((LlapAwareSplit)split).canUseLlapIo();
+    }
+    if (!useLlapIo) {
+      LlapIoImpl.LOG.warn("Not using LLAP IO for an unsupported split: " + split);
+      @SuppressWarnings("unchecked")
+      RecordReader<NullWritable, VectorizedRowBatch> rr =
+          sourceInputFormat.getRecordReader(split, job, reporter);
+      return rr;
+    }
     boolean isVectorMode = Utilities.isVectorMode(job);
     if (!isVectorMode) {
-      LlapIoImpl.LOG.error("No llap in non-vectorized mode");
-      throw new UnsupportedOperationException("No llap in non-vectorized mode");
+      LlapIoImpl.LOG.error("No LLAP IO in non-vectorized mode");
+      throw new UnsupportedOperationException("No LLAP IO in non-vectorized mode");
     }
     FileSplit fileSplit = (FileSplit)split;
     reporter.setStatus(fileSplit.toString());
@@ -102,7 +120,7 @@ public class LlapInputFormat
 
   private class LlapRecordReader
       implements RecordReader<NullWritable, VectorizedRowBatch>, Consumer<ColumnVectorBatch> {
-    private final InputSplit split;
+    private final FileSplit split;
     private final List<Integer> columnIds;
     private final SearchArgument sarg;
     private final String[] columnNames;
@@ -140,7 +158,7 @@ public class LlapInputFormat
       int partitionColumnCount = rbCtx.getPartitionColumnCount();
       if (partitionColumnCount > 0) {
         partitionValues = new Object[partitionColumnCount];
-        rbCtx.getPartitionValues(rbCtx, job, split, partitionValues);
+        VectorizedRowBatchCtx.getPartitionValues(rbCtx, job, split, partitionValues);
       } else {
         partitionValues = null;
       }
@@ -328,4 +346,9 @@ public class LlapInputFormat
       return 0.0f;
     }
   }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return sourceASC == null ? false : sourceASC.shouldSkipCombine(path, conf);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java
index 5c34fa3..b3b571d 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/ColumnVectorProducer.java
@@ -24,13 +24,13 @@ import org.apache.hadoop.hive.llap.counters.QueryFragmentCounters;
 import org.apache.hadoop.hive.llap.io.api.impl.ColumnVectorBatch;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.FileSplit;
 
 /**
  * Entry point used by LlapInputFormat to create read pipeline to get data.
  */
 public interface ColumnVectorProducer {
-  ReadPipeline createReadPipeline(Consumer<ColumnVectorBatch> consumer, InputSplit split,
+  ReadPipeline createReadPipeline(Consumer<ColumnVectorBatch> consumer, FileSplit split,
       List<Integer> columnIds, SearchArgument sarg, String[] columnNames,
       QueryFragmentCounters counters);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
index 259c483..e156eaa 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hive.llap.metrics.LlapDaemonQueueMetrics;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer;
 import org.apache.hadoop.hive.ql.io.orc.encoded.OrcCacheKey;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.FileSplit;
 
 public class OrcColumnVectorProducer implements ColumnVectorProducer {
 
@@ -65,7 +65,7 @@ public class OrcColumnVectorProducer implements ColumnVectorProducer {
 
   @Override
   public ReadPipeline createReadPipeline(
-      Consumer<ColumnVectorBatch> consumer, InputSplit split,
+      Consumer<ColumnVectorBatch> consumer, FileSplit split,
       List<Integer> columnIds, SearchArgument sarg, String[] columnNames,
       QueryFragmentCounters counters) {
     cacheMetrics.incrCacheReadRequests();

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
index 5957d57..58d2ac8 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
@@ -161,14 +161,14 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
   private volatile boolean isPaused = false;
 
   public OrcEncodedDataReader(LowLevelCache lowLevelCache, Cache<OrcCacheKey> cache,
-      OrcMetadataCache metadataCache, Configuration conf, InputSplit split,
+      OrcMetadataCache metadataCache, Configuration conf, FileSplit split,
       List<Integer> columnIds, SearchArgument sarg, String[] columnNames,
       OrcEncodedDataConsumer consumer, QueryFragmentCounters counters) {
     this.lowLevelCache = lowLevelCache;
     this.metadataCache = metadataCache;
     this.cache = cache;
     this.conf = conf;
-    this.split = (FileSplit)split;
+    this.split = split;
     this.columnIds = columnIds;
     if (this.columnIds != null) {
       Collections.sort(this.columnIds);

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index 24137e2..f248054 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.shims.HadoopShims.HdfsFileStatusWithId;
@@ -36,6 +37,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.regex.Pattern;
 
 /**
@@ -655,4 +658,20 @@ public class AcidUtils {
       }
     }
   }
+
+  public static boolean isTablePropertyTransactional(Properties props) {
+    String resultStr = props.getProperty(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL);
+    if (resultStr == null) {
+      resultStr = props.getProperty(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.toUpperCase());
+    }
+    return resultStr != null && resultStr.equalsIgnoreCase("true");
+  }
+
+  public static boolean isTablePropertyTransactional(Map<String, String> parameters) {
+    String resultStr = parameters.get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL);
+    if (resultStr == null) {
+      resultStr = parameters.get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.toUpperCase());
+    }
+    return resultStr != null && resultStr.equalsIgnoreCase("true");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/ql/src/java/org/apache/hadoop/hive/ql/io/LlapAwareSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/LlapAwareSplit.java b/ql/src/java/org/apache/hadoop/hive/ql/io/LlapAwareSplit.java
new file mode 100644
index 0000000..ead4678
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/LlapAwareSplit.java
@@ -0,0 +1,27 @@
+/**
+ * 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.io;
+
+/**
+ * Split that is aware that it could be executed in LLAP. Allows LlapInputFormat to do
+ * a last-minute check to see of LLAP IO pipeline should be used for this particular split.
+ * By default, there is no such check - whatever is sent in is attempted with LLAP IO.
+ */
+public interface LlapAwareSplit {
+  boolean canUseLlapIo();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
index 61cde41..76f1328 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java
@@ -31,6 +31,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.io.ColumnarSplit;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
+import org.apache.hadoop.hive.ql.io.LlapAwareSplit;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.mapred.FileSplit;
@@ -41,9 +42,7 @@ import org.apache.hadoop.mapred.FileSplit;
  * OrcFileSplit. Holds file meta info
  *
  */
-public class OrcSplit extends FileSplit implements ColumnarSplit {
-  private static final Logger LOG = LoggerFactory.getLogger(OrcSplit.class);
-
+public class OrcSplit extends FileSplit implements ColumnarSplit, LlapAwareSplit {
   private FileMetaInfo fileMetaInfo;
   private boolean hasFooter;
   private boolean isOriginal;
@@ -58,7 +57,7 @@ public class OrcSplit extends FileSplit implements ColumnarSplit {
   static final int ORIGINAL_FLAG = 2;
   static final int FOOTER_FLAG = 1;
 
-  protected OrcSplit(){
+  protected OrcSplit() {
     //The FileSplit() constructor in hadoop 0.20 and 1.x is package private so can't use it.
     //This constructor is used to create the object and then call readFields()
     // so just pass nulls to this super constructor.
@@ -186,4 +185,16 @@ public class OrcSplit extends FileSplit implements ColumnarSplit {
   public long getColumnarProjectionSize() {
     return projColsUncompressedSize;
   }
+
+  @Override
+  public boolean canUseLlapIo() {
+    return isOriginal && (deltas == null || deltas.isEmpty());
+  }
+
+  @Override
+  public String toString() {
+    return "OrcSplit [" + getPath() + ", start=" + getStart() + ", length=" + getLength()
+        + ", isOriginal=" + isOriginal + ", hasBase=" + hasBase + ", deltas="
+        + (deltas == null ? 0 : deltas.size()) + "]";
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 5131951..2433cae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -12190,12 +12190,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   public static boolean isAcidTable(Table tab) {
     if (tab == null) return false;
     if (!SessionState.get().getTxnMgr().supportsAcid()) return false;
-    String tableIsTransactional =
-        tab.getProperty(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL);
-    if(tableIsTransactional == null) {
-      tableIsTransactional = tab.getProperty(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.toUpperCase());
-    }
-    return tableIsTransactional != null && tableIsTransactional.equalsIgnoreCase("true");
+    return AcidUtils.isTablePropertyTransactional(tab.getParameters());
   }
 
   private boolean isAcidOutputFormat(Class<? extends OutputFormat> of) {

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
index 73e8f6d..a9d46bf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorUtils;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol;
 import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.SortCol;
@@ -59,7 +60,7 @@ import com.google.common.collect.Interner;
  * This class is also used in the explain command any property with the
  * appropriate annotation will be displayed in the explain output.
  */
-@SuppressWarnings({"serial", "deprecation"})
+@SuppressWarnings({"serial"})
 public class MapWork extends BaseWork {
 
   // use LinkedHashMap to make sure the iteration order is
@@ -205,22 +206,37 @@ public class MapWork extends BaseWork {
   }
 
   public void deriveLlap(Configuration conf) {
-    boolean hasLlap = false, hasNonLlap = false;
+    boolean hasLlap = false, hasNonLlap = false, hasAcid = false;
     boolean isLlapOn = HiveInputFormat.isLlapEnabled(conf),
         canWrapAny = isLlapOn && HiveInputFormat.canWrapAnyForLlap(conf, this);
     boolean hasPathToPartInfo = (pathToPartitionInfo != null && !pathToPartitionInfo.isEmpty());
     if (canWrapAny && hasPathToPartInfo) {
+      assert isLlapOn;
       for (PartitionDesc part : pathToPartitionInfo.values()) {
-        boolean isUsingLlapIo = isLlapOn
-            && HiveInputFormat.canWrapForLlap(part.getInputFileFormatClass());
-        hasLlap |= isUsingLlapIo;
-        hasNonLlap |= (!isUsingLlapIo);
+        boolean isUsingLlapIo = HiveInputFormat.canWrapForLlap(part.getInputFileFormatClass());
+        if (isUsingLlapIo) {
+          if (part.getTableDesc() != null &&
+              AcidUtils.isTablePropertyTransactional(part.getTableDesc().getProperties())) {
+            hasAcid = true;
+          } else {
+            hasLlap = true;
+          }
+        } else {
+          hasNonLlap = true;
+        }
       }
-    } else {
-      hasNonLlap = true;
     }
-    llapIoDesc = isLlapOn ? (canWrapAny ? (hasPathToPartInfo ? ((hasLlap == hasNonLlap) ?
-        "some inputs" : (hasLlap ? "all inputs" : "no inputs")) : "unknown") : "no inputs") : null;
+    llapIoDesc = deriveLlapIoDescString(
+        isLlapOn, canWrapAny, hasPathToPartInfo, hasLlap, hasNonLlap, hasAcid);
+  }
+
+  private static String deriveLlapIoDescString(boolean isLlapOn, boolean canWrapAny,
+      boolean hasPathToPartInfo, boolean hasLlap, boolean hasNonLlap, boolean hasAcid) {
+    if (!isLlapOn) return null; // LLAP IO is off, don't output.
+    if (!canWrapAny) return "no inputs"; // Cannot use with input formats.
+    if (!hasPathToPartInfo) return "unknown"; // No information to judge.
+    if (hasAcid) return "may be used (ACID table)";
+    return (hasLlap ? (hasNonLlap ? "some inputs" : "all inputs") : "no inputs");
   }
 
   public void internTable(Interner<TableDesc> interner) {

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/ql/src/test/queries/clientpositive/llap_acid.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/llap_acid.q b/ql/src/test/queries/clientpositive/llap_acid.q
new file mode 100644
index 0000000..6bd216a
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/llap_acid.q
@@ -0,0 +1,53 @@
+set hive.mapred.mode=nonstrict;
+SET hive.vectorized.execution.enabled=true;
+
+SET hive.llap.io.enabled=false;
+
+SET hive.exec.orc.default.buffer.size=32768;
+SET hive.exec.orc.default.row.index.stride=1000;
+SET hive.optimize.index.filter=true;
+set hive.fetch.task.conversion=none;
+
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+DROP TABLE orc_llap;
+
+CREATE TABLE orc_llap (
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE)
+partitioned by (csmallint smallint)
+clustered by (cint) into 2 buckets stored as orc;
+
+insert into table orc_llap partition (csmallint = 1)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10;
+insert into table orc_llap partition (csmallint = 2)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10;
+
+alter table orc_llap SET TBLPROPERTIES ('transactional'='true');
+
+insert into table orc_llap partition (csmallint = 3)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble desc limit 10;
+
+SET hive.llap.io.enabled=true;
+
+explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint;
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint;
+
+insert into table orc_llap partition (csmallint = 1) values (1, 1, 1, 1);
+
+update orc_llap set cbigint = 2 where cint = 1;
+
+explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint;
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint;
+
+DROP TABLE orc_llap;

http://git-wip-us.apache.org/repos/asf/hive/blob/a53d8af4/ql/src/test/results/clientpositive/llap_acid.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap_acid.q.out b/ql/src/test/results/clientpositive/llap_acid.q.out
new file mode 100644
index 0000000..4ad611d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap_acid.q.out
@@ -0,0 +1,307 @@
+PREHOOK: query: DROP TABLE orc_llap
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE orc_llap
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE orc_llap (
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE)
+partitioned by (csmallint smallint)
+clustered by (cint) into 2 buckets stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@orc_llap
+POSTHOOK: query: CREATE TABLE orc_llap (
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE)
+partitioned by (csmallint smallint)
+clustered by (cint) into 2 buckets stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@orc_llap
+PREHOOK: query: insert into table orc_llap partition (csmallint = 1)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: query: insert into table orc_llap partition (csmallint = 1)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cbigint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cdouble SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cfloat SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+PREHOOK: query: insert into table orc_llap partition (csmallint = 2)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@orc_llap@csmallint=2
+POSTHOOK: query: insert into table orc_llap partition (csmallint = 2)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble asc limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@orc_llap@csmallint=2
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=2).cbigint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=2).cdouble SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=2).cfloat SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=2).cint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+PREHOOK: query: alter table orc_llap SET TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: ALTERTABLE_PROPERTIES
+PREHOOK: Input: default@orc_llap
+PREHOOK: Output: default@orc_llap
+POSTHOOK: query: alter table orc_llap SET TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: ALTERTABLE_PROPERTIES
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Output: default@orc_llap
+PREHOOK: query: insert into table orc_llap partition (csmallint = 3)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble desc limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@orc_llap@csmallint=3
+POSTHOOK: query: insert into table orc_llap partition (csmallint = 3)
+select cint, cbigint, cfloat, cdouble from alltypesorc order by cdouble desc limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@orc_llap@csmallint=3
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=3).cbigint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=3).cdouble SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=3).cfloat SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=3).cint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+PREHOOK: query: explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: orc_llap
+            filterExpr: cint is not null (type: boolean)
+            Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: cint is not null (type: boolean)
+              Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: cint (type: int), csmallint (type: smallint), cbigint (type: bigint)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: int)
+                  sort order: ++
+                  Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col2 (type: bigint)
+      Execution mode: vectorized
+      LLAP IO: may be used (ACID table)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey1 (type: int), KEY.reducesinkkey0 (type: smallint), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 30 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_llap
+PREHOOK: Input: default@orc_llap@csmallint=1
+PREHOOK: Input: default@orc_llap@csmallint=2
+PREHOOK: Input: default@orc_llap@csmallint=3
+#### A masked pattern was here ####
+POSTHOOK: query: select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Input: default@orc_llap@csmallint=1
+POSTHOOK: Input: default@orc_llap@csmallint=2
+POSTHOOK: Input: default@orc_llap@csmallint=3
+#### A masked pattern was here ####
+-285355633	1	-1241163445
+-109813638	1	-58941842
+164554497	1	1161977292
+199879534	1	123351087
+246423894	1	-1645852809
+354670578	1	562841852
+455419170	1	1108177470
+665801232	1	480783141
+708885482	1	-1645852809
+-285355633	2	-1241163445
+-109813638	2	-58941842
+164554497	2	1161977292
+199879534	2	123351087
+246423894	2	-1645852809
+354670578	2	562841852
+455419170	2	1108177470
+665801232	2	480783141
+708885482	2	-1645852809
+-923308739	3	-1887561756
+-3728	3	-1887561756
+762	3	-1645852809
+6981	3	-1887561756
+253665376	3	NULL
+497728223	3	-1887561756
+528534767	3	NULL
+528534767	3	NULL
+528534767	3	NULL
+528534767	3	NULL
+PREHOOK: query: insert into table orc_llap partition (csmallint = 1) values (1, 1, 1, 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: query: insert into table orc_llap partition (csmallint = 1) values (1, 1, 1, 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cbigint EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cdouble EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cfloat EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: orc_llap PARTITION(csmallint=1).cint EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: update orc_llap set cbigint = 2 where cint = 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_llap
+PREHOOK: Input: default@orc_llap@csmallint=1
+PREHOOK: Input: default@orc_llap@csmallint=2
+PREHOOK: Input: default@orc_llap@csmallint=3
+PREHOOK: Output: default@orc_llap@csmallint=1
+PREHOOK: Output: default@orc_llap@csmallint=2
+PREHOOK: Output: default@orc_llap@csmallint=3
+POSTHOOK: query: update orc_llap set cbigint = 2 where cint = 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Input: default@orc_llap@csmallint=1
+POSTHOOK: Input: default@orc_llap@csmallint=2
+POSTHOOK: Input: default@orc_llap@csmallint=3
+POSTHOOK: Output: default@orc_llap@csmallint=1
+POSTHOOK: Output: default@orc_llap@csmallint=2
+POSTHOOK: Output: default@orc_llap@csmallint=3
+PREHOOK: query: explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: orc_llap
+            filterExpr: cint is not null (type: boolean)
+            Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: cint is not null (type: boolean)
+              Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: cint (type: int), csmallint (type: smallint), cbigint (type: bigint)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: int)
+                  sort order: ++
+                  Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col2 (type: bigint)
+      Execution mode: vectorized
+      LLAP IO: may be used (ACID table)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey1 (type: int), KEY.reducesinkkey0 (type: smallint), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 31 Data size: 416 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_llap
+PREHOOK: Input: default@orc_llap@csmallint=1
+PREHOOK: Input: default@orc_llap@csmallint=2
+PREHOOK: Input: default@orc_llap@csmallint=3
+#### A masked pattern was here ####
+POSTHOOK: query: select cint, csmallint, cbigint from orc_llap where cint is not null order
+by csmallint, cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Input: default@orc_llap@csmallint=1
+POSTHOOK: Input: default@orc_llap@csmallint=2
+POSTHOOK: Input: default@orc_llap@csmallint=3
+#### A masked pattern was here ####
+-285355633	1	-1241163445
+-109813638	1	-58941842
+1	1	2
+164554497	1	1161977292
+199879534	1	123351087
+246423894	1	-1645852809
+354670578	1	562841852
+455419170	1	1108177470
+665801232	1	480783141
+708885482	1	-1645852809
+-285355633	2	-1241163445
+-109813638	2	-58941842
+164554497	2	1161977292
+199879534	2	123351087
+246423894	2	-1645852809
+354670578	2	562841852
+455419170	2	1108177470
+665801232	2	480783141
+708885482	2	-1645852809
+-923308739	3	-1887561756
+-3728	3	-1887561756
+762	3	-1645852809
+6981	3	-1887561756
+253665376	3	NULL
+497728223	3	-1887561756
+528534767	3	NULL
+528534767	3	NULL
+528534767	3	NULL
+528534767	3	NULL
+PREHOOK: query: DROP TABLE orc_llap
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@orc_llap
+PREHOOK: Output: default@orc_llap
+POSTHOOK: query: DROP TABLE orc_llap
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@orc_llap
+POSTHOOK: Output: default@orc_llap