This is an automated email from the ASF dual-hosted git repository.

sankarh pushed a commit to branch branch-3
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/branch-3 by this push:
     new b1d1550f8b9 HIVE-27644: Backport of HIVE-17917, HIVE-21457, HIVE-22582 
into branch-3 (Aman Raj, reviewed by Sankar Hariappan)
b1d1550f8b9 is described below

commit b1d1550f8b9d2b6488fb8222fcaa0bf5fdb70179
Author: Aman Raj <104416558+amanraj2...@users.noreply.github.com>
AuthorDate: Sat Sep 16 14:04:54 2023 +0530

    HIVE-27644: Backport of HIVE-17917, HIVE-21457, HIVE-22582 into branch-3 
(Aman Raj, reviewed by Sankar Hariappan)
    
    * HIVE-17917: VectorizedOrcAcidRowBatchReader.computeOffsetAndBucket 
optimization (Saurabh Seth via Eugene Koifman)
    (cherry picked from commit 34331f3c7b69200a0177f5446f1f15c8ed69ee86)
    Resolved merge conflict in VectorizedOrcAcidRowBatchReader.java
    
    * HIVE-21457: Perf optimizations in ORC split-generation (Prasanth 
Jayachandran reviewed by Gopal V)
    (cherry picked from commit 72d72d4df734ccc653a0a6986c319200dea35f0b)
    Resolved conflicts in AcidUtils.java, CompactorMR.java and 
OrcInputFormat.java
    
    * HIVE-22582: Avoid reading table as ACID when table name is starting with 
"delta" , but table is not transactional and BI Split Strategy is used (Aditya 
Shah reviewed by Laszlo Pinter and Peter Vary)
    (cherry picked from commit e6ef2826879fbb9b3ec7987255dda8ec14831a05)
    
    Signed-off-by: Sankar Hariappan <sank...@apache.org>
    Closes (#4686)
---
 .../apache/hadoop/hive/ql/exec/FetchOperator.java  |   2 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java    |  16 ++-
 .../hive/ql/io/HiveContextAwareRecordReader.java   |   5 +-
 .../apache/hadoop/hive/ql/io/HiveInputFormat.java  |   2 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java      |  83 ++++++++++--
 .../hadoop/hive/ql/io/orc/OrcRawRecordMerger.java  |   5 +-
 .../org/apache/hadoop/hive/ql/io/orc/OrcSplit.java |  58 +++++++-
 .../ql/io/orc/VectorizedOrcAcidRowBatchReader.java |  57 ++++----
 .../hadoop/hive/ql/txn/compactor/CompactorMR.java  |   4 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java    |   2 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java      |  18 ++-
 .../clientpositive/acid_vectorization_original.q   |  29 +++-
 .../llap/acid_vectorization_original.q.out         | 146 +++++++++++++++++++++
 13 files changed, 359 insertions(+), 68 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
index 183fae5b9d4..223e52b88d3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
@@ -745,7 +745,7 @@ public class FetchOperator implements Serializable {
   private FileStatus[] listStatusUnderPath(FileSystem fs, Path p) throws 
IOException {
     boolean recursive = job.getBoolean(FileInputFormat.INPUT_DIR_RECURSIVE, 
false);
     // If this is in acid format always read it recursively regardless of what 
the jobconf says.
-    if (!recursive && !AcidUtils.isAcid(p, job)) {
+    if (!recursive && !AcidUtils.isAcid(fs, p, job)) {
       return fs.listStatus(p, FileUtils.HIDDEN_FILES_PATH_FILTER);
     }
     List<FileStatus> results = new ArrayList<FileStatus>();
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 0257801df41..f47c0433f59 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
@@ -915,14 +915,15 @@ public class AcidUtils {
 
   /**
    * Is the given directory in ACID format?
+   * @param fileSystem file system instance
    * @param directory the partition directory to check
    * @param conf the query configuration
    * @return true, if it is an ACID directory
    * @throws IOException
    */
-  public static boolean isAcid(Path directory,
+  public static boolean isAcid(FileSystem fileSystem, Path directory,
                                Configuration conf) throws IOException {
-    FileSystem fs = directory.getFileSystem(conf);
+    FileSystem fs = fileSystem == null ? directory.getFileSystem(conf) : 
fileSystem;
     for(FileStatus file: fs.listStatus(directory)) {
       String filename = file.getPath().getName();
       if (filename.startsWith(BASE_PREFIX) ||
@@ -941,7 +942,7 @@ public class AcidUtils {
       Configuration conf,
       ValidWriteIdList writeIdList
       ) throws IOException {
-    return getAcidState(directory, conf, writeIdList, false, false);
+    return getAcidState(null, directory, conf, writeIdList, false, false);
   }
 
   /** State class for getChildState; cannot modify 2 things in a method. */
@@ -957,28 +958,29 @@ public class AcidUtils {
    * base and diff directories. Note that because major compactions don't
    * preserve the history, we can't use a base directory that includes a
    * write id that we must exclude.
+   * @param fileSystem file system instance
    * @param directory the partition directory to analyze
    * @param conf the configuration
    * @param writeIdList the list of write ids that we are reading
    * @return the state of the directory
    * @throws IOException
    */
-  public static Directory getAcidState(Path directory,
+  public static Directory getAcidState(FileSystem fileSystem, Path directory,
                                        Configuration conf,
                                        ValidWriteIdList writeIdList,
                                        boolean useFileIds,
                                        boolean ignoreEmptyFiles
                                        ) throws IOException {
-    return getAcidState(directory, conf, writeIdList, Ref.from(useFileIds), 
ignoreEmptyFiles, null);
+    return getAcidState(fileSystem, directory, conf, writeIdList, 
Ref.from(useFileIds), ignoreEmptyFiles, null);
   }
 
-  public static Directory getAcidState(Path directory,
+  public static Directory getAcidState(FileSystem fileSystem, Path directory,
                                        Configuration conf,
                                        ValidWriteIdList writeIdList,
                                        Ref<Boolean> useFileIds,
                                        boolean ignoreEmptyFiles,
                                        Map<String, String> tblproperties) 
throws IOException {
-    FileSystem fs = directory.getFileSystem(conf);
+    FileSystem fs = fileSystem == null ? directory.getFileSystem(conf) : 
fileSystem;
     // The following 'deltas' includes all kinds of delta files including 
insert & delete deltas.
     final List<ParsedDelta> deltas = new ArrayList<ParsedDelta>();
     List<ParsedDelta> working = new ArrayList<ParsedDelta>();
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
index 60833bf8ea4..a0148f7e9fe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
@@ -187,8 +187,8 @@ public abstract class HiveContextAwareRecordReader<K, V> 
implements RecordReader
     long blockStart = -1;
     FileSplit fileSplit = split;
     Path path = fileSplit.getPath();
-    FileSystem fs = path.getFileSystem(job);
     if (inputFormatClass.getName().contains("SequenceFile")) {
+      FileSystem fs = path.getFileSystem(job);
       SequenceFile.Reader in = new SequenceFile.Reader(fs, path, job);
       blockPointer = in.isBlockCompressed();
       in.sync(fileSplit.getStart());
@@ -198,6 +198,7 @@ public abstract class HiveContextAwareRecordReader<K, V> 
implements RecordReader
       blockPointer = true;
       blockStart = ((RCFileRecordReader) recordReader).getStart();
     } else if (inputFormatClass.getName().contains("RCFile")) {
+      FileSystem fs = path.getFileSystem(job);
       blockPointer = true;
       RCFile.Reader in = new RCFile.Reader(fs, path, job);
       in.sync(fileSplit.getStart());
@@ -205,7 +206,7 @@ public abstract class HiveContextAwareRecordReader<K, V> 
implements RecordReader
       in.close();
     }
     this.jobConf = job;
-    this.initIOContext(blockStart, blockPointer, path.makeQualified(fs));
+    this.initIOContext(blockStart, blockPointer, path);
 
     this.initIOContextSortedProps(split, recordReader, job);
   }
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 0b613168f91..f617c3ca01e 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
@@ -614,7 +614,7 @@ public class HiveInputFormat<K extends WritableComparable, 
V extends Writable>
     }
     if (hasAcidDirs) {
       AcidUtils.Directory dirInfo = AcidUtils.getAcidState(
-          dir, conf, validWriteIdList, Ref.from(false), true, null);
+          fs, dir, conf, validWriteIdList, Ref.from(false), true, null);
 
       // Find the base, created for IOW.
       Path base = dirInfo.getBaseDirectory();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index f2f93e07322..aa46e36868f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -48,6 +48,9 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hive.common.BlobStorageUtils;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -193,7 +196,7 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
   @Override
   public boolean shouldSkipCombine(Path path,
                                    Configuration conf) throws IOException {
-    return (conf.get(AcidUtils.CONF_ACID_KEY) != null) || 
AcidUtils.isAcid(path, conf);
+    return (conf.get(AcidUtils.CONF_ACID_KEY) != null) || 
AcidUtils.isAcid(null, path, conf);
   }
 
 
@@ -625,6 +628,8 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
     private SplitStrategyKind splitStrategyKind;
     private final SearchArgument sarg;
     private final AcidOperationalProperties acidOperationalProperties;
+    private final boolean isAcid;
+    private final boolean isVectorMode;
 
     Context(Configuration conf) throws IOException {
       this(conf, 1, null);
@@ -638,6 +643,8 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
     Context(Configuration conf, final int minSplits, 
ExternalFooterCachesByConf efc)
         throws IOException {
       this.conf = conf;
+      this.isAcid = AcidUtils.isFullAcidScan(conf);
+      this.isVectorMode = Utilities.getIsVectorized(conf);
       this.forceThreadpool = HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST);
       this.sarg = ConvertAstToSearchArg.createFromConf(conf);
       minSize = HiveConf.getLongVar(conf, ConfVars.MAPREDMINSPLITSIZE, 
DEFAULT_MIN_SPLIT_SIZE);
@@ -716,8 +723,37 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
 
       String value = conf.get(ValidWriteIdList.VALID_WRITEIDS_KEY);
       writeIdList = value == null ? new ValidReaderWriteIdList() : new 
ValidReaderWriteIdList(value);
-      LOG.debug("Context:: Read ValidWriteIdList: " + writeIdList.toString()
-              + " isTransactionalTable: " + isTxnTable + " properties: " + 
txnProperties);
+      LOG.info("Context:: " +
+          "isAcid: {} " +
+          "isVectorMode: {} " +
+          "sarg: {} " +
+          "minSplitSize: {} " +
+          "maxSplitSize: {} " +
+          "splitStrategy: {} " +
+          "footerInSplits: {} " +
+          "numBuckets: {} " +
+          "numThreads: {} " +
+          "cacheMemSize: {} " +
+          "cacheStripeDetails: {} " +
+          "useSoftReference: {} " +
+          "writeIdList: {} " +
+          "isTransactionalTable: {} " +
+          "txnProperties: {} ",
+        isAcid,
+        isVectorMode,
+        sarg,
+        minSize,
+        maxSize,
+        splitStrategyKind,
+        footerInSplits,
+        numBuckets,
+        numThreads,
+        cacheMemSize,
+        cacheStripeDetails,
+        useSoftReference,
+        writeIdList,
+        isTxnTable,
+        txnProperties);
     }
 
     @VisibleForTesting
@@ -1036,6 +1072,9 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
     private final Path dir;
     private final boolean allowSyntheticFileIds;
     private final boolean isDefaultFs;
+    private final Configuration conf;
+    private final boolean isAcid;
+    private final boolean vectorMode;
 
     /**
      * @param dir - root of partition dir
@@ -1051,14 +1090,25 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
       this.dir = dir;
       this.allowSyntheticFileIds = allowSyntheticFileIds;
       this.isDefaultFs = isDefaultFs;
+      this.conf = context.conf;
+      this.isAcid = context.isAcid;
+      this.vectorMode = context.isVectorMode;
     }
 
     @Override
     public List<OrcSplit> getSplits() throws IOException {
       List<OrcSplit> splits = Lists.newArrayList();
+      boolean isAcid = AcidUtils.isFullAcidScan(conf);
+      boolean vectorMode = Utilities.getIsVectorized(conf);
+      OrcSplit.OffsetAndBucketProperty offsetAndBucket = null;
       for (HdfsFileStatusWithId file : fileStatuses) {
+        if (isOriginal && isAcid && vectorMode) {
+          offsetAndBucket = 
VectorizedOrcAcidRowBatchReader.computeOffsetAndBucket(file.getFileStatus(), 
dir,
+              isOriginal, !deltas.isEmpty(), conf);
+        }
+
         FileStatus fileStatus = file.getFileStatus();
-        long logicalLen = AcidUtils.getLogicalLength(fs, fileStatus);
+        long logicalLen = isAcid ? AcidUtils.getLogicalLength(fs, fileStatus) 
: fileStatus.getLen();
         if (logicalLen != 0) {
           Object fileKey = isDefaultFs ? file.getFileId() : null;
           if (fileKey == null && allowSyntheticFileIds) {
@@ -1072,7 +1122,7 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
             }
             OrcSplit orcSplit = new OrcSplit(fileStatus.getPath(), fileKey, 
entry.getKey(),
                 entry.getValue().getLength(), entry.getValue().getHosts(), 
null, isOriginal, true,
-                deltas, -1, logicalLen, dir);
+                deltas, -1, logicalLen, dir, offsetAndBucket);
             splits.add(orcSplit);
           }
         }
@@ -1205,7 +1255,7 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
       }
       //todo: shouldn't ignoreEmptyFiles be set based on ExecutionEngine?
       AcidUtils.Directory dirInfo = AcidUtils.getAcidState(
-          dir, context.conf, context.writeIdList, useFileIds, true, null);
+          fs, dir, context.conf, context.writeIdList, useFileIds, true, null);
       // find the base files (original or new style)
       List<AcidBaseFileInfo> baseFiles = new ArrayList<>();
       if (dirInfo.getBaseDirectory() == null) {
@@ -1352,6 +1402,7 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
     private SchemaEvolution evolution;
     //this is the root of the partition in which the 'file' is located
     private final Path rootDir;
+    OrcSplit.OffsetAndBucketProperty offsetAndBucket = null;
 
     public SplitGenerator(SplitInfo splitInfo, UserGroupInformation ugi,
         boolean allowSyntheticFileIds, boolean isDefaultFs) throws IOException 
{
@@ -1480,7 +1531,7 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
         fileKey = new SyntheticFileId(file);
       }
       return new OrcSplit(file.getPath(), fileKey, offset, length, hosts,
-          orcTail, isOriginal, hasBase, deltas, scaledProjSize, fileLen, 
rootDir);
+          orcTail, isOriginal, hasBase, deltas, scaledProjSize, fileLen, 
rootDir, offsetAndBucket);
     }
 
     private static final class OffsetAndLength { // Java cruft; pair of long.
@@ -1519,6 +1570,14 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
     }
 
     private List<OrcSplit> callInternal() throws IOException {
+      boolean isAcid = context.isAcid;
+      boolean vectorMode = context.isVectorMode;
+
+      if (isOriginal && isAcid && vectorMode) {
+        offsetAndBucket = 
VectorizedOrcAcidRowBatchReader.computeOffsetAndBucket(file, rootDir, 
isOriginal,
+            !deltas.isEmpty(), context.conf);
+      }
+
       // Figure out which stripes we need to read.
       if (ppdResult != null) {
         assert deltaSplits.isEmpty();
@@ -1931,9 +1990,8 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
   @Override
   public InputSplit[] getSplits(JobConf job,
                                 int numSplits) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("getSplits started");
-    }
+    long start = System.currentTimeMillis();
+    LOG.info("getSplits started");
     Configuration conf = job;
     if (HiveConf.getBoolVar(job, 
HiveConf.ConfVars.HIVE_ORC_MS_FOOTER_CACHE_ENABLED)) {
       // Create HiveConf once, since this is expensive.
@@ -1941,9 +1999,8 @@ public class OrcInputFormat implements 
InputFormat<NullWritable, OrcStruct>,
     }
     List<OrcSplit> result = generateSplitsInfo(conf,
         new Context(conf, numSplits, createExternalCaches()));
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("getSplits finished");
-    }
+    long end = System.currentTimeMillis();
+    LOG.info("getSplits finished (#splits: {}). duration: {} ms", 
result.size(), (end - start));
     return result.toArray(new InputSplit[result.size()]);
   }
 
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
index 4ccc0285069..a6c445bd995 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
@@ -463,7 +463,8 @@ public class OrcRawRecordMerger implements 
AcidInputFormat.RawReader<OrcStruct>{
          */
         //the split is from something other than the 1st file of the logical 
bucket - compute offset
         AcidUtils.Directory directoryState
-                = AcidUtils.getAcidState(mergerOptions.getRootPath(), conf, 
validWriteIdList, false, true);
+                = AcidUtils.getAcidState(null, mergerOptions.getRootPath(), 
conf, validWriteIdList, false,
+          true);
         for (HadoopShims.HdfsFileStatusWithId f : 
directoryState.getOriginalFiles()) {
           int bucketIdFromPath = 
AcidUtils.parseBucketId(f.getFileStatus().getPath());
           if (bucketIdFromPath != bucketId) {
@@ -577,7 +578,7 @@ public class OrcRawRecordMerger implements 
AcidInputFormat.RawReader<OrcStruct>{
       assert options.getOffset() == 0;
       assert options.getMaxOffset() == Long.MAX_VALUE;
       AcidUtils.Directory directoryState
-              = AcidUtils.getAcidState(mergerOptions.getRootPath(), conf, 
validWriteIdList, false, true);
+              = AcidUtils.getAcidState(null, mergerOptions.getRootPath(), 
conf, validWriteIdList, false, true);
       /**
        * Note that for reading base_x/ or delta_x_x/ with non-acid schema,
        * {@link Options#getRootPath()} is set to base_x/ or delta_x_x/ which 
causes all it's
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 64428f0d2ad..9df7860b3fb 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
@@ -64,6 +64,11 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
   private transient Object fileKey;
   private long fileLen;
 
+  static final int HAS_SYNTHETIC_ACID_PROPS_FLAG = 32;
+  /**
+   * This contains the synthetic ROW__ID offset and bucket properties for 
original file splits in an ACID table.
+   */
+  private OffsetAndBucketProperty syntheticAcidProps;
   static final int HAS_SYNTHETIC_FILEID_FLAG = 16;
   static final int HAS_LONG_FILEID_FLAG = 8;
   static final int BASE_FLAG = 4;
@@ -79,7 +84,8 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
 
   public OrcSplit(Path path, Object fileId, long offset, long length, String[] 
hosts,
       OrcTail orcTail, boolean isOriginal, boolean hasBase,
-      List<AcidInputFormat.DeltaMetaData> deltas, long projectedDataSize, long 
fileLen, Path rootDir) {
+      List<AcidInputFormat.DeltaMetaData> deltas, long projectedDataSize, long 
fileLen, Path rootDir,
+      OffsetAndBucketProperty syntheticAcidProps) {
     super(path, offset, length, hosts);
     // For HDFS, we could avoid serializing file ID and just replace the path 
with inode-based
     // path. However, that breaks bunch of stuff because Hive later looks up 
things by split path.
@@ -93,6 +99,7 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
     this.projColsUncompressedSize = projectedDataSize <= 0 ? length : 
projectedDataSize;
     // setting file length to Long.MAX_VALUE will let orc reader read file 
length from file system
     this.fileLen = fileLen <= 0 ? Long.MAX_VALUE : fileLen;
+    this.syntheticAcidProps = syntheticAcidProps;
   }
 
   @Override
@@ -120,7 +127,8 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
         (isOriginal ? ORIGINAL_FLAG : 0) |
         (hasFooter ? FOOTER_FLAG : 0) |
         (isFileIdLong ? HAS_LONG_FILEID_FLAG : 0) |
-        (isFileIdWritable ? HAS_SYNTHETIC_FILEID_FLAG : 0);
+        (isFileIdWritable ? HAS_SYNTHETIC_FILEID_FLAG : 0) |
+        (syntheticAcidProps != null? HAS_SYNTHETIC_ACID_PROPS_FLAG : 0);
     out.writeByte(flags);
     out.writeInt(deltas.size());
     for(AcidInputFormat.DeltaMetaData delta: deltas) {
@@ -140,6 +148,11 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
     }
     out.writeLong(fileLen);
     out.writeUTF(rootDir.toString());
+    if (syntheticAcidProps != null) {
+      out.writeLong(syntheticAcidProps.rowIdOffset);
+      out.writeInt(syntheticAcidProps.bucketProperty);
+      out.writeLong(syntheticAcidProps.syntheticWriteId);
+    }
   }
 
   @Override
@@ -152,7 +165,8 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
     isOriginal = (ORIGINAL_FLAG & flags) != 0;
     hasBase = (BASE_FLAG & flags) != 0;
     boolean hasLongFileId = (HAS_LONG_FILEID_FLAG & flags) != 0,
-        hasWritableFileId = (HAS_SYNTHETIC_FILEID_FLAG & flags) != 0;
+        hasWritableFileId = (HAS_SYNTHETIC_FILEID_FLAG & flags) != 0,
+        hasSyntheticProps = (HAS_SYNTHETIC_ACID_PROPS_FLAG & flags) != 0;
     if (hasLongFileId && hasWritableFileId) {
       throw new IOException("Invalid split - both file ID types present");
     }
@@ -180,6 +194,14 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
     }
     fileLen = in.readLong();
     rootDir = new Path(in.readUTF());
+
+    if (hasSyntheticProps) {
+      long rowId = in.readLong();
+      int bucket = in.readInt();
+      long writeId = in.readLong();
+
+      syntheticAcidProps = new OffsetAndBucketProperty(rowId, bucket, writeId);
+    }
   }
 
   public OrcTail getOrcTail() {
@@ -234,6 +256,10 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
     return fileKey;
   }
 
+  public OffsetAndBucketProperty getSyntheticAcidProps() {
+    return syntheticAcidProps;
+  }
+
   @Override
   public long getColumnarProjectionSize() {
     return projColsUncompressedSize;
@@ -275,6 +301,32 @@ public class OrcSplit extends FileSplit implements 
ColumnarSplit, LlapAwareSplit
     return false;
   }
 
+  /**
+   * Used for generating synthetic ROW__IDs for reading "original" files.
+   */
+  static final class OffsetAndBucketProperty {
+    private final long rowIdOffset;
+    private final int bucketProperty;
+    private final long syntheticWriteId;
+    OffsetAndBucketProperty(long rowIdOffset, int bucketProperty, long 
syntheticWriteId) {
+      this.rowIdOffset = rowIdOffset;
+      this.bucketProperty = bucketProperty;
+      this.syntheticWriteId = syntheticWriteId;
+    }
+
+    public long getRowIdOffset() {
+      return rowIdOffset;
+    }
+
+    public int getBucketProperty() {
+      return bucketProperty;
+    }
+
+    public long getSyntheticWriteId() {
+      return syntheticWriteId;
+    }
+  }
+
   @Override
   public String toString() {
     return "OrcSplit [" + getPath() + ", start=" + getStart() + ", length=" + 
getLength()
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcAcidRowBatchReader.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcAcidRowBatchReader.java
index 1841cfaa2e7..34a12af515e 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcAcidRowBatchReader.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/VectorizedOrcAcidRowBatchReader.java
@@ -27,6 +27,7 @@ import java.util.Map.Entry;
 import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
@@ -94,7 +95,7 @@ public class VectorizedOrcAcidRowBatchReader
   /**
    * for reading "original" files
    */
-  private final OffsetAndBucketProperty syntheticProps;
+  private final OrcSplit.OffsetAndBucketProperty syntheticProps;
   /**
    * To have access to {@link RecordReader#getRowNumber()} in the underlying 
file
    */
@@ -229,7 +230,7 @@ public class VectorizedOrcAcidRowBatchReader
     rowIdProjected = areRowIdsProjected(rbCtx);
     rootPath = orcSplit.getRootDir();
     //why even compute syntheticProps if !isOriginal???
-    syntheticProps = computeOffsetAndBucket(orcSplit, conf, validWriteIdList);
+    syntheticProps = orcSplit.getSyntheticAcidProps();
   }
 
   public void setBaseAndInnerReader(
@@ -258,15 +259,15 @@ public class VectorizedOrcAcidRowBatchReader
    * When reading a split of an "original" file and we need to decorate data 
with ROW__ID.
    * This requires treating multiple files that are part of the same bucket 
(tranche for unbucketed
    * tables) as a single logical file to number rowids consistently.
-   *
-   * todo: This logic is executed per split of every "original" file.  The 
computed result is the
-   * same for every split form the same file so this could be optimized by 
moving it to
-   * before/during split computation and passing the info in the split.  
(HIVE-17917)
    */
-  private OffsetAndBucketProperty computeOffsetAndBucket(
-      OrcSplit split, JobConf conf, ValidWriteIdList validWriteIdList) throws 
IOException {
-    if (!needSyntheticRowIds(split.isOriginal(), 
!deleteEventRegistry.isEmpty(), rowIdProjected)) {
-      if(split.isOriginal()) {
+  static OrcSplit.OffsetAndBucketProperty computeOffsetAndBucket(
+          FileStatus file, Path rootDir, boolean isOriginal, boolean 
hasDeletes,
+          Configuration conf) throws IOException {
+
+    VectorizedRowBatchCtx vrbCtx = Utilities.getVectorizedRowBatchCtx(conf);
+
+    if (!needSyntheticRowIds(isOriginal, hasDeletes, 
areRowIdsProjected(vrbCtx))) {
+      if(isOriginal) {
         /**
          * Even if we don't need to project ROW_IDs, we still need to check 
the write ID that
          * created the file to see if it's committed.  See more in
@@ -274,27 +275,32 @@ public class VectorizedOrcAcidRowBatchReader
          * filter out base/delta files but this makes fewer dependencies)
          */
         OrcRawRecordMerger.TransactionMetaData syntheticTxnInfo =
-            
OrcRawRecordMerger.TransactionMetaData.findWriteIDForSynthetcRowIDs(split.getPath(),
-                    split.getRootDir(), conf);
-        return new OffsetAndBucketProperty(-1,-1, 
syntheticTxnInfo.syntheticWriteId);
+            
OrcRawRecordMerger.TransactionMetaData.findWriteIDForSynthetcRowIDs(file.getPath(),
+                    rootDir, conf);
+        return new OrcSplit.OffsetAndBucketProperty(-1, -1, 
syntheticTxnInfo.syntheticWriteId);
       }
       return null;
     }
+
+    String txnString = conf.get(ValidWriteIdList.VALID_WRITEIDS_KEY);
+    ValidWriteIdList validWriteIdList = (txnString == null) ? new 
ValidReaderWriteIdList() :
+        new ValidReaderWriteIdList(txnString);
+
     long rowIdOffset = 0;
     OrcRawRecordMerger.TransactionMetaData syntheticTxnInfo =
-        
OrcRawRecordMerger.TransactionMetaData.findWriteIDForSynthetcRowIDs(split.getPath(),
 split.getRootDir(), conf);
-    int bucketId = AcidUtils.parseBucketId(split.getPath());
+        
OrcRawRecordMerger.TransactionMetaData.findWriteIDForSynthetcRowIDs(file.getPath(),
 rootDir, conf);
+    int bucketId = AcidUtils.parseBucketId(file.getPath());
     int bucketProperty = BucketCodec.V1.encode(new 
AcidOutputFormat.Options(conf)
         //statementId is from directory name (or 0 if there is none)
       .statementId(syntheticTxnInfo.statementId).bucket(bucketId));
-    AcidUtils.Directory directoryState = AcidUtils.getAcidState( 
syntheticTxnInfo.folder, conf,
+    AcidUtils.Directory directoryState = AcidUtils.getAcidState(null, 
syntheticTxnInfo.folder, conf,
         validWriteIdList, false, true);
     for (HadoopShims.HdfsFileStatusWithId f : 
directoryState.getOriginalFiles()) {
       int bucketIdFromPath = 
AcidUtils.parseBucketId(f.getFileStatus().getPath());
       if (bucketIdFromPath != bucketId) {
         continue;//HIVE-16952
       }
-      if (f.getFileStatus().getPath().equals(split.getPath())) {
+      if (f.getFileStatus().getPath().equals(file.getPath())) {
         //'f' is the file whence this split is
         break;
       }
@@ -302,7 +308,7 @@ public class VectorizedOrcAcidRowBatchReader
         OrcFile.readerOptions(conf));
       rowIdOffset += reader.getNumberOfRows();
     }
-    return new OffsetAndBucketProperty(rowIdOffset, bucketProperty,
+    return new OrcSplit.OffsetAndBucketProperty(rowIdOffset, bucketProperty,
       syntheticTxnInfo.syntheticWriteId);
   }
   /**
@@ -499,8 +505,9 @@ public class VectorizedOrcAcidRowBatchReader
     boolean needSyntheticRowId =
         needSyntheticRowIds(true, !deleteEventRegistry.isEmpty(), 
rowIdProjected);
     if(needSyntheticRowId) {
-      assert syntheticProps != null && syntheticProps.rowIdOffset >= 0 : "" + 
syntheticProps;
-      assert syntheticProps != null && syntheticProps.bucketProperty >= 0 : "" 
+ syntheticProps;
+      assert syntheticProps != null : "" + syntheticProps;
+      assert syntheticProps.getRowIdOffset() >= 0 : "" + syntheticProps;
+      assert syntheticProps.getBucketProperty() >= 0 : "" + syntheticProps;
       if(innerReader == null) {
         throw new IllegalStateException(getClass().getName() + " requires " +
           org.apache.orc.RecordReader.class +
@@ -511,14 +518,14 @@ public class VectorizedOrcAcidRowBatchReader
        */
       recordIdColumnVector.fields[0].noNulls = true;
       recordIdColumnVector.fields[0].isRepeating = true;
-      ((LongColumnVector)recordIdColumnVector.fields[0]).vector[0] = 
syntheticProps.syntheticWriteId;
+      ((LongColumnVector)recordIdColumnVector.fields[0]).vector[0] = 
syntheticProps.getSyntheticWriteId();
       /**
        * This is {@link RecordIdentifier#getBucketProperty()}
        * Also see {@link BucketCodec}
        */
       recordIdColumnVector.fields[1].noNulls = true;
       recordIdColumnVector.fields[1].isRepeating = true;
-      ((LongColumnVector)recordIdColumnVector.fields[1]).vector[0] = 
syntheticProps.bucketProperty;
+      ((LongColumnVector)recordIdColumnVector.fields[1]).vector[0] = 
syntheticProps.getBucketProperty();
       /**
        * {@link RecordIdentifier#getRowId()}
        */
@@ -527,7 +534,7 @@ public class VectorizedOrcAcidRowBatchReader
       long[] rowIdVector = 
((LongColumnVector)recordIdColumnVector.fields[2]).vector;
       for(int i = 0; i < vectorizedRowBatchBase.size; i++) {
         //baseReader.getRowNumber() seems to point at the start of the batch 
todo: validate
-        rowIdVector[i] = syntheticProps.rowIdOffset + 
innerReader.getRowNumber() + i;
+        rowIdVector[i] = syntheticProps.getRowIdOffset() + 
innerReader.getRowNumber() + i;
       }
       //Now populate a structure to use to apply delete events
       innerRecordIdColumnVector = new ColumnVector[OrcRecordUpdater.FIELDS];
@@ -537,7 +544,7 @@ public class VectorizedOrcAcidRowBatchReader
       //these are insert events so (original txn == current) txn for all rows
       innerRecordIdColumnVector[OrcRecordUpdater.CURRENT_WRITEID] = 
recordIdColumnVector.fields[0];
     }
-    if(syntheticProps.syntheticWriteId > 0) {
+    if(syntheticProps.getSyntheticWriteId() > 0) {
       //"originals" (written before table was converted to acid) is considered 
written by
       // writeid:0 which is always committed so there is no need to check wrt 
invalid write Ids
       //But originals written by Load Data for example can be in base_x or 
delta_x_x so we must
@@ -551,7 +558,7 @@ public class VectorizedOrcAcidRowBatchReader
         * reader (transactions) is concerned.  Since here we are reading 
'original' schema file,
         * all rows in it have been created by the same txn, namely 
'syntheticProps.syntheticWriteId'
         */
-        if (!validWriteIdList.isWriteIdValid(syntheticProps.syntheticWriteId)) 
{
+        if 
(!validWriteIdList.isWriteIdValid(syntheticProps.getSyntheticWriteId())) {
           selectedBitSet.clear(0, vectorizedRowBatchBase.size);
         }
       }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java 
b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
index e3ceb3af055..745b5ed0c4d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
@@ -251,7 +251,7 @@ public class CompactorMR {
     // and discovering that in getSplits is too late as we then have no way to 
pass it to our
     // mapper.
 
-    AcidUtils.Directory dir = AcidUtils.getAcidState(new 
Path(sd.getLocation()), conf, writeIds, false, true);
+    AcidUtils.Directory dir = AcidUtils.getAcidState(null, new 
Path(sd.getLocation()), conf, writeIds, false, true);
     List<AcidUtils.ParsedDelta> parsedDeltas = dir.getCurrentDirectories();
     int maxDeltastoHandle = 
conf.getIntVar(HiveConf.ConfVars.COMPACTOR_MAX_NUM_DELTA);
     if(parsedDeltas.size() > maxDeltastoHandle) {
@@ -326,7 +326,7 @@ public class CompactorMR {
       StorageDescriptor sd, ValidWriteIdList writeIds, CompactionInfo ci) 
throws IOException {
     LOG.debug("Going to delete directories for aborted transactions for MM 
table "
         + t.getDbName() + "." + t.getTableName());
-    AcidUtils.Directory dir = AcidUtils.getAcidState(new 
Path(sd.getLocation()),
+    AcidUtils.Directory dir = AcidUtils.getAcidState(null, new 
Path(sd.getLocation()),
         conf, writeIds, Ref.from(false), false, t.getParameters());
     removeFilesForMmTable(conf, dir);
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index a61b6e8d370..7fbc71b7042 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -269,7 +269,7 @@ public class Initiator extends CompactorThread {
     boolean noBase = false;
     Path location = new Path(sd.getLocation());
     FileSystem fs = location.getFileSystem(conf);
-    AcidUtils.Directory dir = AcidUtils.getAcidState(location, conf, writeIds, 
false, false);
+    AcidUtils.Directory dir = AcidUtils.getAcidState(fs, location, conf, 
writeIds, false, false);
     Path base = dir.getBaseDirectory();
     long baseSize = 0;
     FileStatus stat = null;
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java 
b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index 326c7f65bfe..0ca29859d6f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -2875,10 +2875,8 @@ public class TestInputOutputFormat {
         readOpsDelta = statistics.getReadOps() - readOpsBefore;
       }
     }
-    // call-1: listLocatedStatus - mock:/mocktbl
-    // call-2: check existence of side file for mock:/mocktbl/0_0
-    // call-3: check existence of side file for  mock:/mocktbl/0_1
-    assertEquals(3, readOpsDelta);
+    // call-1: getAcidState - mock:/mocktbl
+    assertEquals(1, readOpsDelta);
 
     // enable cache and use default strategy
     conf.set(ConfVars.HIVE_ORC_CACHE_STRIPE_DETAILS_MEMORY_SIZE.varname, 
"10Mb");
@@ -2916,7 +2914,7 @@ public class TestInputOutputFormat {
         readOpsDelta = statistics.getReadOps() - readOpsBefore;
       }
     }
-    // call-1: listLocatedStatus - mock:/mocktbl
+    // call-1: getAcidState - mock:/mocktbl
     assertEquals(1, readOpsDelta);
 
     // revert back to local fs
@@ -3031,7 +3029,7 @@ public class TestInputOutputFormat {
         readOpsDelta = statistics.getReadOps() - readOpsBefore;
       }
     }
-    // call-1: listLocatedStatus - mock:/mocktbl1
+    // call-1: getAcidState - mock:/mocktbl1
     assertEquals(1, readOpsDelta);
 
     // revert back to local fs
@@ -3150,7 +3148,7 @@ public class TestInputOutputFormat {
         readOpsDelta = statistics.getReadOps() - readOpsBefore;
       }
     }
-    // call-1: listLocatedStatus - mock:/mocktbl2
+    // call-1: getAcidState - mock:/mocktbl2
     assertEquals(1, readOpsDelta);
 
     // revert back to local fs
@@ -4030,7 +4028,7 @@ public class TestInputOutputFormat {
     conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0,2");
     OrcSplit split = new OrcSplit(testFilePath, null, 0, fileLength,
         new String[0], null, false, true,
-        new ArrayList<AcidInputFormat.DeltaMetaData>(), fileLength, 
fileLength, workDir);
+        new ArrayList<AcidInputFormat.DeltaMetaData>(), fileLength, 
fileLength, workDir, null);
     OrcInputFormat inputFormat = new OrcInputFormat();
     AcidInputFormat.RowReader<OrcStruct> reader = inputFormat.getReader(split,
         new AcidInputFormat.Options(conf));
@@ -4058,7 +4056,7 @@ public class TestInputOutputFormat {
     conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0,2,3");
     split = new OrcSplit(testFilePath, null, 0, fileLength,
         new String[0], null, false, true,
-        new ArrayList<AcidInputFormat.DeltaMetaData>(), fileLength, 
fileLength, workDir);
+        new ArrayList<AcidInputFormat.DeltaMetaData>(), fileLength, 
fileLength, workDir, null);
     inputFormat = new OrcInputFormat();
     reader = inputFormat.getReader(split, new AcidInputFormat.Options(conf));
     record = 0;
@@ -4176,7 +4174,7 @@ public class TestInputOutputFormat {
     // Specify an OrcSplit that starts beyond the offset of the last stripe.
     OrcSplit split = new OrcSplit(testFilePath, null, lastStripeOffset + 1, 
lastStripeLength,
         new String[0], null, false, true,
-        new ArrayList<AcidInputFormat.DeltaMetaData>(), fileLength, 
fileLength, workDir);
+        new ArrayList<AcidInputFormat.DeltaMetaData>(), fileLength, 
fileLength, workDir, null);
     OrcInputFormat inputFormat = new OrcInputFormat();
     AcidInputFormat.RowReader<OrcStruct> reader = inputFormat.getReader(split,
         new AcidInputFormat.Options(conf));
diff --git a/ql/src/test/queries/clientpositive/acid_vectorization_original.q 
b/ql/src/test/queries/clientpositive/acid_vectorization_original.q
index 5082aedf902..9decbec7482 100644
--- a/ql/src/test/queries/clientpositive/acid_vectorization_original.q
+++ b/ql/src/test/queries/clientpositive/acid_vectorization_original.q
@@ -132,4 +132,31 @@ select ROW__ID, * from over10k_orc_bucketed where ROW__ID 
is null;
 -- this test that there are no duplicate ROW__IDs so should produce no output
 -- select ROW__ID, count(*) from over10k_orc_bucketed group by ROW__ID having 
count(*) > 1;
 
--- select ROW__ID, * from over10k_orc_bucketed where ROW__ID is null;
\ No newline at end of file
+-- select ROW__ID, * from over10k_orc_bucketed where ROW__ID is null;
+
+CREATE TABLE over10k_orc STORED AS ORC as select * from over10k_n2 where t 
between 3 and 4;
+-- Make sure there are multiple original files
+INSERT INTO over10k_orc select * from over10k_n2 where t between 3 and 4;
+alter table over10k_orc set TBLPROPERTIES ('transactional'='true');
+
+-- row id is projected but there are no delete deltas
+set hive.exec.orc.split.strategy=ETL;
+select o1.ROW__ID r1, o1.* from over10k_orc o1 join over10k_orc o2
+on o1.ROW__ID.rowid == o2.ROW__ID.rowid and o1.ROW__ID.writeid == 
o2.ROW__ID.writeid and o1.ROW__ID.bucketid == o2.ROW__ID.bucketid;
+
+set hive.exec.orc.split.strategy=BI;
+select o1.ROW__ID r1, o1.* from over10k_orc o1 join over10k_orc o2
+on o1.ROW__ID.rowid == o2.ROW__ID.rowid
+and o1.ROW__ID.writeid == o2.ROW__ID.writeid
+and o1.ROW__ID.bucketid == o2.ROW__ID.bucketid;
+
+delete from over10k_orc where t = 3;
+
+-- row id not projected but has delete deltas
+set hive.exec.orc.split.strategy=ETL;
+select t, count(*) from over10k_orc
+group by t;
+
+set hive.exec.orc.split.strategy=BI;
+select t, count(*) from over10k_orc
+group by t;
diff --git 
a/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out 
b/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
index a23c4b16a91..511530a56e6 100644
--- a/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
@@ -743,3 +743,149 @@ POSTHOOK: query: select ROW__ID, * from 
over10k_orc_bucketed where ROW__ID is nu
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@over10k_orc_bucketed
 #### A masked pattern was here ####
+PREHOOK: query: CREATE TABLE over10k_orc STORED AS ORC as select * from 
over10k_n2 where t between 3 and 4
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@over10k_n2
+PREHOOK: Output: database:default
+PREHOOK: Output: default@over10k_orc
+POSTHOOK: query: CREATE TABLE over10k_orc STORED AS ORC as select * from 
over10k_n2 where t between 3 and 4
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@over10k_n2
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@over10k_orc
+POSTHOOK: Lineage: over10k_orc.b SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.bin SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.bo SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.d SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.dec SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:dec, type:decimal(4,2), comment:null), 
]
+POSTHOOK: Lineage: over10k_orc.f SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.i SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.s SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.si SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.t SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.ts SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: INSERT INTO over10k_orc select * from over10k_n2 where t 
between 3 and 4
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k_n2
+PREHOOK: Output: default@over10k_orc
+POSTHOOK: query: INSERT INTO over10k_orc select * from over10k_n2 where t 
between 3 and 4
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k_n2
+POSTHOOK: Output: default@over10k_orc
+POSTHOOK: Lineage: over10k_orc.b SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:b, type:bigint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.bin SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:bin, type:binary, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.bo SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:bo, type:boolean, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.d SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.dec SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:dec, type:decimal(4,2), comment:null), 
]
+POSTHOOK: Lineage: over10k_orc.f SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:f, type:float, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.i SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.s SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:s, type:string, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.si SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:si, type:smallint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.t SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:t, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: over10k_orc.ts SIMPLE 
[(over10k_n2)over10k_n2.FieldSchema(name:ts, type:timestamp, comment:null), ]
+PREHOOK: query: alter table over10k_orc set TBLPROPERTIES 
('transactional'='true')
+PREHOOK: type: ALTERTABLE_PROPERTIES
+PREHOOK: Input: default@over10k_orc
+PREHOOK: Output: default@over10k_orc
+POSTHOOK: query: alter table over10k_orc set TBLPROPERTIES 
('transactional'='true')
+POSTHOOK: type: ALTERTABLE_PROPERTIES
+POSTHOOK: Input: default@over10k_orc
+POSTHOOK: Output: default@over10k_orc
+PREHOOK: query: select o1.ROW__ID r1, o1.* from over10k_orc o1 join 
over10k_orc o2
+on o1.ROW__ID.rowid == o2.ROW__ID.rowid and o1.ROW__ID.writeid == 
o2.ROW__ID.writeid and o1.ROW__ID.bucketid == o2.ROW__ID.bucketid
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: select o1.ROW__ID r1, o1.* from over10k_orc o1 join 
over10k_orc o2
+on o1.ROW__ID.rowid == o2.ROW__ID.rowid and o1.ROW__ID.writeid == 
o2.ROW__ID.writeid and o1.ROW__ID.bucketid == o2.ROW__ID.bucketid
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k_orc
+#### A masked pattern was here ####
+{"writeid":0,"bucketid":536870912,"rowid":0}   3       260     65659   
4294967508      91.53   43.18   false   oscar ovid      2013-03-01 
09:11:58.703281      99.87   chemistry
+{"writeid":0,"bucketid":536870912,"rowid":1}   4       279     65745   
4294967431      83.58   31.66   true    fred van buren  2013-03-01 
09:11:58.703087      25.19   study skills
+{"writeid":0,"bucketid":536870912,"rowid":2}   3       469     65743   
4294967428      10.66   39.84   false   victor zipper   2013-03-01 
09:11:58.703181      26.60   mathematics
+{"writeid":0,"bucketid":536870912,"rowid":3}   4       392     65665   
4294967391      53.27   3.86    true    zach miller     2013-03-01 
09:11:58.703296      43.66   undecided
+{"writeid":0,"bucketid":536870912,"rowid":4}   3       395     65747   
4294967313      57.25   3.17    true    wendy garcia    2013-03-01 
09:11:58.703074      58.47   xylophone band
+{"writeid":0,"bucketid":536870912,"rowid":5}   3       277     65788   
4294967403      58.08   20.55   false   xavier ovid     2013-03-01 
09:11:58.703281      62.11   zync studies
+{"writeid":0,"bucketid":536870912,"rowid":6}   4       509     65776   
4294967432      78.26   35.02   false   mike king       2013-03-01 
09:11:58.703231      18.70   undecided
+{"writeid":0,"bucketid":536870912,"rowid":7}   3       308     65757   
4294967430      49.28   38.04   false   nick zipper     2013-03-01 
09:11:58.703132      1.86    kindergarten
+{"writeid":0,"bucketid":536870912,"rowid":8}   4       460     65625   
4294967360      5.51    22.6    true    oscar laertes   2013-03-01 
09:11:58.703293      42.86   nap time
+{"writeid":0,"bucketid":536870912,"rowid":9}   3       322     65672   
4294967508      25.55   26.28   true    jessica carson  2013-03-01 
09:11:58.70312       52.60   education
+{"writeid":0,"bucketid":536870912,"rowid":10}  3       464     65617   
4294967424      82.3    2.92    false   ethan brown     2013-03-01 
09:11:58.703076      18.51   wind surfing
+{"writeid":0,"bucketid":536870912,"rowid":11}  3       260     65659   
4294967508      91.53   43.18   false   oscar ovid      2013-03-01 
09:11:58.703281      99.87   chemistry
+{"writeid":0,"bucketid":536870912,"rowid":12}  4       279     65745   
4294967431      83.58   31.66   true    fred van buren  2013-03-01 
09:11:58.703087      25.19   study skills
+{"writeid":0,"bucketid":536870912,"rowid":13}  3       469     65743   
4294967428      10.66   39.84   false   victor zipper   2013-03-01 
09:11:58.703181      26.60   mathematics
+{"writeid":0,"bucketid":536870912,"rowid":14}  4       392     65665   
4294967391      53.27   3.86    true    zach miller     2013-03-01 
09:11:58.703296      43.66   undecided
+{"writeid":0,"bucketid":536870912,"rowid":15}  3       395     65747   
4294967313      57.25   3.17    true    wendy garcia    2013-03-01 
09:11:58.703074      58.47   xylophone band
+{"writeid":0,"bucketid":536870912,"rowid":16}  3       277     65788   
4294967403      58.08   20.55   false   xavier ovid     2013-03-01 
09:11:58.703281      62.11   zync studies
+{"writeid":0,"bucketid":536870912,"rowid":17}  4       509     65776   
4294967432      78.26   35.02   false   mike king       2013-03-01 
09:11:58.703231      18.70   undecided
+{"writeid":0,"bucketid":536870912,"rowid":18}  3       308     65757   
4294967430      49.28   38.04   false   nick zipper     2013-03-01 
09:11:58.703132      1.86    kindergarten
+{"writeid":0,"bucketid":536870912,"rowid":19}  4       460     65625   
4294967360      5.51    22.6    true    oscar laertes   2013-03-01 
09:11:58.703293      42.86   nap time
+{"writeid":0,"bucketid":536870912,"rowid":20}  3       322     65672   
4294967508      25.55   26.28   true    jessica carson  2013-03-01 
09:11:58.70312       52.60   education
+{"writeid":0,"bucketid":536870912,"rowid":21}  3       464     65617   
4294967424      82.3    2.92    false   ethan brown     2013-03-01 
09:11:58.703076      18.51   wind surfing
+PREHOOK: query: select o1.ROW__ID r1, o1.* from over10k_orc o1 join 
over10k_orc o2
+on o1.ROW__ID.rowid == o2.ROW__ID.rowid
+and o1.ROW__ID.writeid == o2.ROW__ID.writeid
+and o1.ROW__ID.bucketid == o2.ROW__ID.bucketid
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: select o1.ROW__ID r1, o1.* from over10k_orc o1 join 
over10k_orc o2
+on o1.ROW__ID.rowid == o2.ROW__ID.rowid
+and o1.ROW__ID.writeid == o2.ROW__ID.writeid
+and o1.ROW__ID.bucketid == o2.ROW__ID.bucketid
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k_orc
+#### A masked pattern was here ####
+{"writeid":0,"bucketid":536870912,"rowid":0}   3       260     65659   
4294967508      91.53   43.18   false   oscar ovid      2013-03-01 
09:11:58.703281      99.87   chemistry
+{"writeid":0,"bucketid":536870912,"rowid":1}   4       279     65745   
4294967431      83.58   31.66   true    fred van buren  2013-03-01 
09:11:58.703087      25.19   study skills
+{"writeid":0,"bucketid":536870912,"rowid":2}   3       469     65743   
4294967428      10.66   39.84   false   victor zipper   2013-03-01 
09:11:58.703181      26.60   mathematics
+{"writeid":0,"bucketid":536870912,"rowid":3}   4       392     65665   
4294967391      53.27   3.86    true    zach miller     2013-03-01 
09:11:58.703296      43.66   undecided
+{"writeid":0,"bucketid":536870912,"rowid":4}   3       395     65747   
4294967313      57.25   3.17    true    wendy garcia    2013-03-01 
09:11:58.703074      58.47   xylophone band
+{"writeid":0,"bucketid":536870912,"rowid":5}   3       277     65788   
4294967403      58.08   20.55   false   xavier ovid     2013-03-01 
09:11:58.703281      62.11   zync studies
+{"writeid":0,"bucketid":536870912,"rowid":6}   4       509     65776   
4294967432      78.26   35.02   false   mike king       2013-03-01 
09:11:58.703231      18.70   undecided
+{"writeid":0,"bucketid":536870912,"rowid":7}   3       308     65757   
4294967430      49.28   38.04   false   nick zipper     2013-03-01 
09:11:58.703132      1.86    kindergarten
+{"writeid":0,"bucketid":536870912,"rowid":8}   4       460     65625   
4294967360      5.51    22.6    true    oscar laertes   2013-03-01 
09:11:58.703293      42.86   nap time
+{"writeid":0,"bucketid":536870912,"rowid":9}   3       322     65672   
4294967508      25.55   26.28   true    jessica carson  2013-03-01 
09:11:58.70312       52.60   education
+{"writeid":0,"bucketid":536870912,"rowid":10}  3       464     65617   
4294967424      82.3    2.92    false   ethan brown     2013-03-01 
09:11:58.703076      18.51   wind surfing
+{"writeid":0,"bucketid":536870912,"rowid":11}  3       260     65659   
4294967508      91.53   43.18   false   oscar ovid      2013-03-01 
09:11:58.703281      99.87   chemistry
+{"writeid":0,"bucketid":536870912,"rowid":12}  4       279     65745   
4294967431      83.58   31.66   true    fred van buren  2013-03-01 
09:11:58.703087      25.19   study skills
+{"writeid":0,"bucketid":536870912,"rowid":13}  3       469     65743   
4294967428      10.66   39.84   false   victor zipper   2013-03-01 
09:11:58.703181      26.60   mathematics
+{"writeid":0,"bucketid":536870912,"rowid":14}  4       392     65665   
4294967391      53.27   3.86    true    zach miller     2013-03-01 
09:11:58.703296      43.66   undecided
+{"writeid":0,"bucketid":536870912,"rowid":15}  3       395     65747   
4294967313      57.25   3.17    true    wendy garcia    2013-03-01 
09:11:58.703074      58.47   xylophone band
+{"writeid":0,"bucketid":536870912,"rowid":16}  3       277     65788   
4294967403      58.08   20.55   false   xavier ovid     2013-03-01 
09:11:58.703281      62.11   zync studies
+{"writeid":0,"bucketid":536870912,"rowid":17}  4       509     65776   
4294967432      78.26   35.02   false   mike king       2013-03-01 
09:11:58.703231      18.70   undecided
+{"writeid":0,"bucketid":536870912,"rowid":18}  3       308     65757   
4294967430      49.28   38.04   false   nick zipper     2013-03-01 
09:11:58.703132      1.86    kindergarten
+{"writeid":0,"bucketid":536870912,"rowid":19}  4       460     65625   
4294967360      5.51    22.6    true    oscar laertes   2013-03-01 
09:11:58.703293      42.86   nap time
+{"writeid":0,"bucketid":536870912,"rowid":20}  3       322     65672   
4294967508      25.55   26.28   true    jessica carson  2013-03-01 
09:11:58.70312       52.60   education
+{"writeid":0,"bucketid":536870912,"rowid":21}  3       464     65617   
4294967424      82.3    2.92    false   ethan brown     2013-03-01 
09:11:58.703076      18.51   wind surfing
+PREHOOK: query: delete from over10k_orc where t = 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k_orc
+PREHOOK: Output: default@over10k_orc
+POSTHOOK: query: delete from over10k_orc where t = 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k_orc
+POSTHOOK: Output: default@over10k_orc
+PREHOOK: query: select t, count(*) from over10k_orc
+group by t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: select t, count(*) from over10k_orc
+group by t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k_orc
+#### A masked pattern was here ####
+4      8
+PREHOOK: query: select t, count(*) from over10k_orc
+group by t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k_orc
+#### A masked pattern was here ####
+POSTHOOK: query: select t, count(*) from over10k_orc
+group by t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k_orc
+#### A masked pattern was here ####
+4      8

Reply via email to