You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2016/08/19 07:09:30 UTC

hive git commit: HIVE-14566: LLAP IO reads timestamp wrongly (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Repository: hive
Updated Branches:
  refs/heads/master 1dc02c134 -> 20a67b334


HIVE-14566: LLAP IO reads timestamp wrongly (Prasanth Jayachandran reviewed by Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: 20a67b334a8178b934b7fe15e4256fd788c4ab9d
Parents: 1dc02c1
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Fri Aug 19 00:09:09 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Fri Aug 19 00:09:09 2016 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   9 +-
 .../llap/io/metadata/OrcStripeMetadata.java     |   8 +
 .../org/apache/orc/impl/TreeReaderFactory.java  |   9 +-
 .../test/org/apache/orc/TestOrcTimezone3.java   | 126 +++
 .../orc/encoded/EncodedTreeReaderFactory.java   |  31 +-
 .../clientpositive/llap/orc_merge12.q.out       | 822 +++++++++++++++++++
 7 files changed, 992 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/20a67b33/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 2c86807..16c09da 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -472,6 +472,7 @@ minillap.shared.query.files=bucket_map_join_tez1.q,\
   llapdecider.q,\
   llap_nullscan.q,\
   mrr.q,\
+  orc_merge12.q,\
   orc_ppd_basic.q,\
   orc_ppd_schema_evol_1a.q,\
   orc_ppd_schema_evol_1b.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/20a67b33/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
index 94e4750..2fa68a9 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
@@ -160,7 +160,8 @@ public class OrcEncodedDataConsumer
       int numCols = batch.getColumnIxs().length;
       if (columnReaders == null || !sameStripe) {
         this.columnReaders = EncodedTreeReaderFactory.createEncodedTreeReader(numCols,
-            fileMetadata.getTypes(), stripeMetadata.getEncodings(), batch, codec, skipCorrupt);
+            fileMetadata.getTypes(), stripeMetadata.getEncodings(), batch, codec, skipCorrupt,
+                stripeMetadata.getWriterTimezone());
         positionInStreams(columnReaders, batch, numCols, stripeMetadata);
       } else {
         repositionInStreams(this.columnReaders, batch, sameStripe, numCols, stripeMetadata);
@@ -224,6 +225,12 @@ public class OrcEncodedDataConsumer
       OrcProto.RowIndex rowIndex = stripeMetadata.getRowIndexes()[columnIndex];
       OrcProto.RowIndexEntry rowIndexEntry = rowIndex.getEntry(rowGroupIndex);
       ((SettableTreeReader)columnReaders[i]).setBuffers(streamBuffers, sameStripe);
+      // TODO: When hive moves to java8, make updateTimezone() as default method in SettableTreeReader so that we can
+      // avoid this check
+      if (columnReaders[i] instanceof EncodedTreeReaderFactory.TimestampStreamReader && !sameStripe) {
+        ((EncodedTreeReaderFactory.TimestampStreamReader) columnReaders[i])
+                .updateTimezone(stripeMetadata.getWriterTimezone());
+      }
       columnReaders[i].seek(new RecordReaderImpl.PositionProviderImpl(rowIndexEntry));
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/20a67b33/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java
index 6874586..bc87094 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java
@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.TimeZone;
+
 import org.apache.hadoop.hive.llap.IncrementalObjectSizeEstimator;
 import org.apache.hadoop.hive.llap.IncrementalObjectSizeEstimator.ObjectEstimator;
 import org.apache.hadoop.hive.llap.cache.EvictionDispatcher;
@@ -37,6 +39,7 @@ public class OrcStripeMetadata extends LlapCacheableBuffer {
   private final OrcBatchKey stripeKey;
   private final List<OrcProto.ColumnEncoding> encodings;
   private final List<OrcProto.Stream> streams;
+  private final String writerTimezone;
   private final long rowCount;
   private OrcIndex rowIndex;
 
@@ -60,6 +63,7 @@ public class OrcStripeMetadata extends LlapCacheableBuffer {
     OrcProto.StripeFooter footer = mr.readStripeFooter(stripe);
     streams = footer.getStreamsList();
     encodings = footer.getColumnsList();
+    writerTimezone = footer.getWriterTimezone();
     rowCount = stripe.getNumberOfRows();
     rowIndex = mr.readRowIndex(stripe, footer, includes, null, sargColumns, null);
 
@@ -70,6 +74,7 @@ public class OrcStripeMetadata extends LlapCacheableBuffer {
     stripeKey = new OrcBatchKey(id, 0, 0);
     encodings = new ArrayList<>();
     streams = new ArrayList<>();
+    writerTimezone = "";
     rowCount = estimatedMemUsage = 0;
   }
 
@@ -123,6 +128,9 @@ public class OrcStripeMetadata extends LlapCacheableBuffer {
     return streams;
   }
 
+  public String getWriterTimezone() {
+    return writerTimezone;
+  }
   @Override
   public long getMemoryUsage() {
     return estimatedMemUsage;

http://git-wip-us.apache.org/repos/asf/hive/blob/20a67b33/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java b/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
index e6fef91..245e3ce 100644
--- a/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
+++ b/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
@@ -815,17 +815,17 @@ public class TreeReaderFactory {
     protected IntegerReader nanos = null;
     private final boolean skipCorrupt;
     private Map<String, Long> baseTimestampMap;
-    private long base_timestamp;
+    protected long base_timestamp;
     private final TimeZone readerTimeZone;
     private TimeZone writerTimeZone;
     private boolean hasSameTZRules;
 
     TimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
-      this(columnId, null, null, null, null, skipCorrupt);
+      this(columnId, null, null, null, null, skipCorrupt, null);
     }
 
     protected TimestampTreeReader(int columnId, InStream presentStream, InStream dataStream,
-        InStream nanosStream, OrcProto.ColumnEncoding encoding, boolean skipCorrupt)
+        InStream nanosStream, OrcProto.ColumnEncoding encoding, boolean skipCorrupt, String writerTimezone)
         throws IOException {
       super(columnId, presentStream);
       this.skipCorrupt = skipCorrupt;
@@ -844,6 +844,7 @@ public class TreeReaderFactory {
         if (nanosStream != null) {
           this.nanos = createIntegerReader(encoding.getKind(), nanosStream, false, skipCorrupt);
         }
+        base_timestamp = getBaseTimestamp(writerTimezone);
       }
     }
 
@@ -870,7 +871,7 @@ public class TreeReaderFactory {
       base_timestamp = getBaseTimestamp(stripeFooter.getWriterTimezone());
     }
 
-    private long getBaseTimestamp(String timeZoneId) throws IOException {
+    protected long getBaseTimestamp(String timeZoneId) throws IOException {
       // to make sure new readers read old files in the same way
       if (timeZoneId == null || timeZoneId.isEmpty()) {
         timeZoneId = readerTimeZone.getID();

http://git-wip-us.apache.org/repos/asf/hive/blob/20a67b33/orc/src/test/org/apache/orc/TestOrcTimezone3.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestOrcTimezone3.java b/orc/src/test/org/apache/orc/TestOrcTimezone3.java
new file mode 100644
index 0000000..40ab0c9
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestOrcTimezone3.java
@@ -0,0 +1,126 @@
+/**
+ * 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.orc;
+
+import static junit.framework.Assert.assertEquals;
+
+import java.io.File;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.google.common.collect.Lists;
+
+import junit.framework.Assert;
+
+/**
+ *
+ */
+@RunWith(Parameterized.class)
+public class TestOrcTimezone3 {
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+  String writerTimeZone;
+  String readerTimeZone;
+  static TimeZone defaultTimeZone = TimeZone.getDefault();
+
+  public TestOrcTimezone3(String writerTZ, String readerTZ) {
+    this.writerTimeZone = writerTZ;
+    this.readerTimeZone = readerTZ;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    List<Object[]> result = Arrays.asList(new Object[][]{
+        {"America/Chicago", "America/Los_Angeles"},
+    });
+    return result;
+  }
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcTimezone3." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @After
+  public void restoreTimeZone() {
+    TimeZone.setDefault(defaultTimeZone);
+  }
+
+  @Test
+  public void testTimestampWriter() throws Exception {
+    TypeDescription schema = TypeDescription.createTimestamp();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(writerTimeZone));
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .bufferSize(10000));
+    assertEquals(writerTimeZone, TimeZone.getDefault().getID());
+    List<String> ts = Lists.newArrayList();
+    ts.add("1969-12-31 16:00:14.007");
+    ts.add("1969-12-31 16:00:06.021");
+    ts.add("1969-12-31 16:00:03.963");
+    VectorizedRowBatch batch = schema.createRowBatch();
+    TimestampColumnVector times = (TimestampColumnVector) batch.cols[0];
+    for (String t : ts) {
+      times.set(batch.size++, Timestamp.valueOf(t));
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    assertEquals(readerTimeZone, TimeZone.getDefault().getID());
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    times = (TimestampColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(ts.get(idx++), times.asScratchTimestamp(r).toString());
+      }
+    }
+    rows.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/20a67b33/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
index b44da06..ebbdf8d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
@@ -39,7 +39,7 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     void setBuffers(ColumnStreamData[] streamBuffers, boolean sameStripe) throws IOException;
   }
 
-  protected static class TimestampStreamReader extends TimestampTreeReader
+  public static class TimestampStreamReader extends TimestampTreeReader
       implements SettableTreeReader {
     private boolean isFileCompressed;
     private SettableUncompressedStream _presentStream;
@@ -47,9 +47,10 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     private SettableUncompressedStream _nanosStream;
 
     private TimestampStreamReader(int columnId, SettableUncompressedStream present,
-        SettableUncompressedStream data, SettableUncompressedStream nanos, boolean isFileCompressed,
-        OrcProto.ColumnEncoding encoding, boolean skipCorrupt) throws IOException {
-      super(columnId, present, data, nanos, encoding, skipCorrupt);
+                                  SettableUncompressedStream data, SettableUncompressedStream nanos,
+                                  boolean isFileCompressed, OrcProto.ColumnEncoding encoding,
+                                  boolean skipCorrupt, String writerTimezoneId) throws IOException {
+      super(columnId, present, data, nanos, encoding, skipCorrupt, writerTimezoneId);
       this.isFileCompressed = isFileCompressed;
       this._presentStream = present;
       this._secondsStream = data;
@@ -96,6 +97,10 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
       }
     }
 
+    public void updateTimezone(String writerTimezoneId) throws IOException {
+      base_timestamp = getBaseTimestamp(writerTimezoneId);
+    }
+
     public static class StreamReaderBuilder {
       private int columnIndex;
       private ColumnStreamData presentStream;
@@ -104,6 +109,7 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
       private CompressionCodec compressionCodec;
       private OrcProto.ColumnEncoding columnEncoding;
       private boolean skipCorrupt;
+      private String writerTimezone;
 
       public StreamReaderBuilder setColumnIndex(int columnIndex) {
         this.columnIndex = columnIndex;
@@ -135,6 +141,11 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
         return this;
       }
 
+      public StreamReaderBuilder setWriterTimezone(String writerTimezoneId) {
+        this.writerTimezone = writerTimezoneId;
+        return this;
+      }
+
       public StreamReaderBuilder skipCorrupt(boolean skipCorrupt) {
         this.skipCorrupt = skipCorrupt;
         return this;
@@ -155,7 +166,7 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
         boolean isFileCompressed = compressionCodec != null;
         return new TimestampStreamReader(columnIndex, present, data, nanos,
-            isFileCompressed, columnEncoding, skipCorrupt);
+            isFileCompressed, columnEncoding, skipCorrupt, writerTimezone);
       }
     }
 
@@ -1663,10 +1674,11 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
   }
 
   public static TreeReader[] createEncodedTreeReader(int numCols,
-      List<OrcProto.Type> types,
-      List<OrcProto.ColumnEncoding> encodings,
-      EncodedColumnBatch<OrcBatchKey> batch,
-      CompressionCodec codec, boolean skipCorrupt) throws IOException {
+                                                     List<OrcProto.Type> types,
+                                                     List<OrcProto.ColumnEncoding> encodings,
+                                                     EncodedColumnBatch<OrcBatchKey> batch,
+                                                     CompressionCodec codec, boolean skipCorrupt,
+                                                     String writerTimezone) throws IOException {
     TreeReader[] treeReaders = new TreeReader[numCols];
     for (int i = 0; i < numCols; i++) {
       int columnIndex = batch.getColumnIxs()[i];
@@ -1814,6 +1826,7 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
               .setNanosStream(secondary)
               .setCompressionCodec(codec)
               .setColumnEncoding(columnEncoding)
+              .setWriterTimezone(writerTimezone)
               .skipCorrupt(skipCorrupt)
               .build();
           break;

http://git-wip-us.apache.org/repos/asf/hive/blob/20a67b33/ql/src/test/results/clientpositive/llap/orc_merge12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/orc_merge12.q.out b/ql/src/test/results/clientpositive/llap/orc_merge12.q.out
new file mode 100644
index 0000000..6a86fcf
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/orc_merge12.q.out
@@ -0,0 +1,822 @@
+PREHOOK: query: CREATE TABLE `alltypesorc3xcols`(
+  `atinyint` tinyint,
+  `asmallint` smallint,
+  `aint` int,
+  `abigint` bigint,
+  `afloat` float,
+  `adouble` double,
+  `astring1` string,
+  `astring2` string,
+  `atimestamp1` timestamp,
+  `atimestamp2` timestamp,
+  `aboolean1` boolean,
+  `aboolean2` boolean,
+  `btinyint` tinyint,
+  `bsmallint` smallint,
+  `bint` int,
+  `bbigint` bigint,
+  `bfloat` float,
+  `bdouble` double,
+  `bstring1` string,
+  `bstring2` string,
+  `btimestamp1` timestamp,
+  `btimestamp2` timestamp,
+  `bboolean1` boolean,
+  `bboolean2` boolean,
+  `ctinyint` tinyint,
+  `csmallint` smallint,
+  `cint` int,
+  `cbigint` bigint,
+  `cfloat` float,
+  `cdouble` double,
+  `cstring1` string,
+  `cstring2` string,
+  `ctimestamp1` timestamp,
+  `ctimestamp2` timestamp,
+  `cboolean1` boolean,
+  `cboolean2` boolean) stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alltypesorc3xcols
+POSTHOOK: query: CREATE TABLE `alltypesorc3xcols`(
+  `atinyint` tinyint,
+  `asmallint` smallint,
+  `aint` int,
+  `abigint` bigint,
+  `afloat` float,
+  `adouble` double,
+  `astring1` string,
+  `astring2` string,
+  `atimestamp1` timestamp,
+  `atimestamp2` timestamp,
+  `aboolean1` boolean,
+  `aboolean2` boolean,
+  `btinyint` tinyint,
+  `bsmallint` smallint,
+  `bint` int,
+  `bbigint` bigint,
+  `bfloat` float,
+  `bdouble` double,
+  `bstring1` string,
+  `bstring2` string,
+  `btimestamp1` timestamp,
+  `btimestamp2` timestamp,
+  `bboolean1` boolean,
+  `bboolean2` boolean,
+  `ctinyint` tinyint,
+  `csmallint` smallint,
+  `cint` int,
+  `cbigint` bigint,
+  `cfloat` float,
+  `cdouble` double,
+  `cstring1` string,
+  `cstring2` string,
+  `ctimestamp1` timestamp,
+  `ctimestamp2` timestamp,
+  `cboolean1` boolean,
+  `cboolean2` boolean) stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alltypesorc3xcols
+PREHOOK: query: load data local inpath '../../data/files/alltypesorc3xcols' into table alltypesorc3xcols
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@alltypesorc3xcols
+POSTHOOK: query: load data local inpath '../../data/files/alltypesorc3xcols' into table alltypesorc3xcols
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@alltypesorc3xcols
+PREHOOK: query: load data local inpath '../../data/files/alltypesorc3xcols' into table alltypesorc3xcols
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@alltypesorc3xcols
+POSTHOOK: query: load data local inpath '../../data/files/alltypesorc3xcols' into table alltypesorc3xcols
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@alltypesorc3xcols
+PREHOOK: query: select count(*) from alltypesorc3xcols
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from alltypesorc3xcols
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+24576
+PREHOOK: query: select sum(hash(*)) from alltypesorc3xcols
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+POSTHOOK: query: select sum(hash(*)) from alltypesorc3xcols
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+-302946892512
+PREHOOK: query: alter table alltypesorc3xcols concatenate
+PREHOOK: type: ALTER_TABLE_MERGE
+PREHOOK: Input: default@alltypesorc3xcols
+PREHOOK: Output: default@alltypesorc3xcols
+POSTHOOK: query: alter table alltypesorc3xcols concatenate
+POSTHOOK: type: ALTER_TABLE_MERGE
+POSTHOOK: Input: default@alltypesorc3xcols
+POSTHOOK: Output: default@alltypesorc3xcols
+PREHOOK: query: select count(*) from alltypesorc3xcols
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from alltypesorc3xcols
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+24576
+PREHOOK: query: select sum(hash(*)) from alltypesorc3xcols
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+POSTHOOK: query: select sum(hash(*)) from alltypesorc3xcols
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+-302946892512
+PREHOOK: query: select * from alltypesorc3xcols limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc3xcols
+#### A masked pattern was here ####
+-- BEGIN ORC FILE DUMP --
+#### A masked pattern was here ####
+File Version: 0.12 with HIVE_13083
+Rows: 24576
+Compression: ZLIB
+Compression size: 262144
+Type: struct<_col0:tinyint,_col1:smallint,_col2:int,_col3:bigint,_col4:float,_col5:double,_col6:string,_col7:string,_col8:timestamp,_col9:timestamp,_col10:boolean,_col11:boolean,_col12:tinyint,_col13:smallint,_col14:int,_col15:bigint,_col16:float,_col17:double,_col18:string,_col19:string,_col20:timestamp,_col21:timestamp,_col22:boolean,_col23:boolean,_col24:tinyint,_col25:smallint,_col26:int,_col27:bigint,_col28:float,_col29:double,_col30:string,_col31:string,_col32:timestamp,_col33:timestamp,_col34:boolean,_col35:boolean>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 12288 hasNull: false
+    Column 1: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
+    Column 2: count: 9174 hasNull: true min: -16379 max: 16376 sum: 7435990
+    Column 3: count: 9173 hasNull: true min: -1073279343 max: 1073680599 sum: 1438050863785
+    Column 4: count: 9173 hasNull: true min: -2147311592 max: 2145498388 sum: -1698460028409
+    Column 5: count: 9173 hasNull: true min: -64.0 max: 79.5530014038086 sum: -39479.635992884636
+    Column 6: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
+    Column 7: count: 12288 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 442368
+    Column 8: count: 12288 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 884736
+    Column 9: count: 9173 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 10: count: 9174 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 11: count: 9174 hasNull: true true: 6138
+    Column 12: count: 9173 hasNull: true true: 3983
+    Column 13: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
+    Column 14: count: 9174 hasNull: true min: -16379 max: 16376 sum: 7435990
+    Column 15: count: 9173 hasNull: true min: -1073279343 max: 1073680599 sum: 1438050863785
+    Column 16: count: 9173 hasNull: true min: -2147311592 max: 2145498388 sum: -1698460028409
+    Column 17: count: 9173 hasNull: true min: -64.0 max: 79.5530014038086 sum: -39479.635992884636
+    Column 18: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
+    Column 19: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
+    Column 20: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
+    Column 21: count: 9173 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 22: count: 9174 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 23: count: 9174 hasNull: true true: 6138
+    Column 24: count: 9173 hasNull: true true: 3983
+    Column 25: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
+    Column 26: count: 9174 hasNull: true min: -16379 max: 16376 sum: 7435990
+    Column 27: count: 9173 hasNull: true min: -1073279343 max: 1073680599 sum: 1438050863785
+    Column 28: count: 9173 hasNull: true min: -2147311592 max: 2145498388 sum: -1698460028409
+    Column 29: count: 9173 hasNull: true min: -64.0 max: 79.5530014038086 sum: -39479.635992884636
+    Column 30: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
+    Column 31: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
+    Column 32: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
+    Column 33: count: 9173 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 34: count: 9174 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 35: count: 9174 hasNull: true true: 6138
+    Column 36: count: 9173 hasNull: true true: 3983
+  Stripe 2:
+    Column 0: count: 12288 hasNull: false
+    Column 1: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
+    Column 2: count: 9174 hasNull: true min: -16379 max: 16376 sum: 7435990
+    Column 3: count: 9173 hasNull: true min: -1073279343 max: 1073680599 sum: 1438050863785
+    Column 4: count: 9173 hasNull: true min: -2147311592 max: 2145498388 sum: -1698460028409
+    Column 5: count: 9173 hasNull: true min: -64.0 max: 79.5530014038086 sum: -39479.635992884636
+    Column 6: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
+    Column 7: count: 12288 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 442368
+    Column 8: count: 12288 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 884736
+    Column 9: count: 9173 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 10: count: 9174 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 11: count: 9174 hasNull: true true: 6138
+    Column 12: count: 9173 hasNull: true true: 3983
+    Column 13: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
+    Column 14: count: 9174 hasNull: true min: -16379 max: 16376 sum: 7435990
+    Column 15: count: 9173 hasNull: true min: -1073279343 max: 1073680599 sum: 1438050863785
+    Column 16: count: 9173 hasNull: true min: -2147311592 max: 2145498388 sum: -1698460028409
+    Column 17: count: 9173 hasNull: true min: -64.0 max: 79.5530014038086 sum: -39479.635992884636
+    Column 18: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
+    Column 19: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
+    Column 20: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
+    Column 21: count: 9173 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 22: count: 9174 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 23: count: 9174 hasNull: true true: 6138
+    Column 24: count: 9173 hasNull: true true: 3983
+    Column 25: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
+    Column 26: count: 9174 hasNull: true min: -16379 max: 16376 sum: 7435990
+    Column 27: count: 9173 hasNull: true min: -1073279343 max: 1073680599 sum: 1438050863785
+    Column 28: count: 9173 hasNull: true min: -2147311592 max: 2145498388 sum: -1698460028409
+    Column 29: count: 9173 hasNull: true min: -64.0 max: 79.5530014038086 sum: -39479.635992884636
+    Column 30: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
+    Column 31: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
+    Column 32: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
+    Column 33: count: 9173 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 34: count: 9174 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+    Column 35: count: 9174 hasNull: true true: 6138
+    Column 36: count: 9173 hasNull: true true: 3983
+
+File Statistics:
+  Column 0: count: 24576 hasNull: false
+  Column 1: count: 18346 hasNull: true min: -64 max: 62 sum: -79712
+  Column 2: count: 18348 hasNull: true min: -16379 max: 16376 sum: 14871980
+  Column 3: count: 18346 hasNull: true min: -1073279343 max: 1073680599 sum: 2876101727570
+  Column 4: count: 18346 hasNull: true min: -2147311592 max: 2145498388 sum: -3396920056818
+  Column 5: count: 18346 hasNull: true min: -64.0 max: 79.5530014038086 sum: -78959.27198576927
+  Column 6: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
+  Column 7: count: 24576 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 884736
+  Column 8: count: 24576 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 1769472
+  Column 9: count: 18346 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+  Column 10: count: 18348 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+  Column 11: count: 18348 hasNull: true true: 12276
+  Column 12: count: 18346 hasNull: true true: 7966
+  Column 13: count: 18346 hasNull: true min: -64 max: 62 sum: -79712
+  Column 14: count: 18348 hasNull: true min: -16379 max: 16376 sum: 14871980
+  Column 15: count: 18346 hasNull: true min: -1073279343 max: 1073680599 sum: 2876101727570
+  Column 16: count: 18346 hasNull: true min: -2147311592 max: 2145498388 sum: -3396920056818
+  Column 17: count: 18346 hasNull: true min: -64.0 max: 79.5530014038086 sum: -78959.27198576927
+  Column 18: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
+  Column 19: count: 18348 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 255762
+  Column 20: count: 18346 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 298268
+  Column 21: count: 18346 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+  Column 22: count: 18348 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+  Column 23: count: 18348 hasNull: true true: 12276
+  Column 24: count: 18346 hasNull: true true: 7966
+  Column 25: count: 18346 hasNull: true min: -64 max: 62 sum: -79712
+  Column 26: count: 18348 hasNull: true min: -16379 max: 16376 sum: 14871980
+  Column 27: count: 18346 hasNull: true min: -1073279343 max: 1073680599 sum: 2876101727570
+  Column 28: count: 18346 hasNull: true min: -2147311592 max: 2145498388 sum: -3396920056818
+  Column 29: count: 18346 hasNull: true min: -64.0 max: 79.5530014038086 sum: -78959.27198576927
+  Column 30: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
+  Column 31: count: 18348 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 255762
+  Column 32: count: 18346 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 298268
+  Column 33: count: 18346 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+  Column 34: count: 18348 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808
+  Column 35: count: 18348 hasNull: true true: 12276
+  Column 36: count: 18346 hasNull: true true: 7966
+
+Stripes:
+  Stripe: offset: 3 data: 1500017 rows: 12288 tail: 501 index: 2836
+    Stream: column 0 section ROW_INDEX start: 3 length 21
+    Stream: column 1 section ROW_INDEX start: 24 length 53
+    Stream: column 2 section ROW_INDEX start: 77 length 67
+    Stream: column 3 section ROW_INDEX start: 144 length 81
+    Stream: column 4 section ROW_INDEX start: 225 length 83
+    Stream: column 5 section ROW_INDEX start: 308 length 77
+    Stream: column 6 section ROW_INDEX start: 385 length 77
+    Stream: column 7 section ROW_INDEX start: 462 length 176
+    Stream: column 8 section ROW_INDEX start: 638 length 267
+    Stream: column 9 section ROW_INDEX start: 905 length 63
+    Stream: column 10 section ROW_INDEX start: 968 length 57
+    Stream: column 11 section ROW_INDEX start: 1025 length 47
+    Stream: column 12 section ROW_INDEX start: 1072 length 47
+    Stream: column 13 section ROW_INDEX start: 1119 length 53
+    Stream: column 14 section ROW_INDEX start: 1172 length 67
+    Stream: column 15 section ROW_INDEX start: 1239 length 81
+    Stream: column 16 section ROW_INDEX start: 1320 length 83
+    Stream: column 17 section ROW_INDEX start: 1403 length 77
+    Stream: column 18 section ROW_INDEX start: 1480 length 77
+    Stream: column 19 section ROW_INDEX start: 1557 length 115
+    Stream: column 20 section ROW_INDEX start: 1672 length 93
+    Stream: column 21 section ROW_INDEX start: 1765 length 63
+    Stream: column 22 section ROW_INDEX start: 1828 length 57
+    Stream: column 23 section ROW_INDEX start: 1885 length 47
+    Stream: column 24 section ROW_INDEX start: 1932 length 47
+    Stream: column 25 section ROW_INDEX start: 1979 length 53
+    Stream: column 26 section ROW_INDEX start: 2032 length 67
+    Stream: column 27 section ROW_INDEX start: 2099 length 81
+    Stream: column 28 section ROW_INDEX start: 2180 length 83
+    Stream: column 29 section ROW_INDEX start: 2263 length 77
+    Stream: column 30 section ROW_INDEX start: 2340 length 77
+    Stream: column 31 section ROW_INDEX start: 2417 length 115
+    Stream: column 32 section ROW_INDEX start: 2532 length 93
+    Stream: column 33 section ROW_INDEX start: 2625 length 63
+    Stream: column 34 section ROW_INDEX start: 2688 length 57
+    Stream: column 35 section ROW_INDEX start: 2745 length 47
+    Stream: column 36 section ROW_INDEX start: 2792 length 47
+    Stream: column 1 section PRESENT start: 2839 length 51
+    Stream: column 1 section DATA start: 2890 length 5448
+    Stream: column 2 section PRESENT start: 8338 length 53
+    Stream: column 2 section DATA start: 8391 length 12144
+    Stream: column 3 section PRESENT start: 20535 length 53
+    Stream: column 3 section DATA start: 20588 length 24618
+    Stream: column 4 section PRESENT start: 45206 length 52
+    Stream: column 4 section DATA start: 45258 length 24681
+    Stream: column 5 section PRESENT start: 69939 length 51
+    Stream: column 5 section DATA start: 69990 length 9927
+    Stream: column 6 section PRESENT start: 79917 length 53
+    Stream: column 6 section DATA start: 79970 length 19755
+    Stream: column 7 section DATA start: 99725 length 258570
+    Stream: column 7 section LENGTH start: 358295 length 108
+    Stream: column 8 section DATA start: 358403 length 517341
+    Stream: column 8 section LENGTH start: 875744 length 108
+    Stream: column 9 section PRESENT start: 875852 length 52
+    Stream: column 9 section DATA start: 875904 length 8045
+    Stream: column 9 section SECONDARY start: 883949 length 9555
+    Stream: column 10 section PRESENT start: 893504 length 58
+    Stream: column 10 section DATA start: 893562 length 8082
+    Stream: column 10 section SECONDARY start: 901644 length 9590
+    Stream: column 11 section PRESENT start: 911234 length 51
+    Stream: column 11 section DATA start: 911285 length 782
+    Stream: column 12 section PRESENT start: 912067 length 54
+    Stream: column 12 section DATA start: 912121 length 783
+    Stream: column 13 section PRESENT start: 912904 length 51
+    Stream: column 13 section DATA start: 912955 length 5448
+    Stream: column 14 section PRESENT start: 918403 length 53
+    Stream: column 14 section DATA start: 918456 length 12144
+    Stream: column 15 section PRESENT start: 930600 length 53
+    Stream: column 15 section DATA start: 930653 length 24618
+    Stream: column 16 section PRESENT start: 955271 length 52
+    Stream: column 16 section DATA start: 955323 length 24681
+    Stream: column 17 section PRESENT start: 980004 length 51
+    Stream: column 17 section DATA start: 980055 length 9927
+    Stream: column 18 section PRESENT start: 989982 length 53
+    Stream: column 18 section DATA start: 990035 length 19755
+    Stream: column 19 section PRESENT start: 1009790 length 51
+    Stream: column 19 section DATA start: 1009841 length 11009
+    Stream: column 19 section LENGTH start: 1020850 length 3722
+    Stream: column 19 section DICTIONARY_DATA start: 1024572 length 65435
+    Stream: column 20 section PRESENT start: 1090007 length 54
+    Stream: column 20 section DATA start: 1090061 length 11006
+    Stream: column 20 section LENGTH start: 1101067 length 3739
+    Stream: column 20 section DICTIONARY_DATA start: 1104806 length 66022
+    Stream: column 21 section PRESENT start: 1170828 length 52
+    Stream: column 21 section DATA start: 1170880 length 8045
+    Stream: column 21 section SECONDARY start: 1178925 length 9555
+    Stream: column 22 section PRESENT start: 1188480 length 58
+    Stream: column 22 section DATA start: 1188538 length 8082
+    Stream: column 22 section SECONDARY start: 1196620 length 9590
+    Stream: column 23 section PRESENT start: 1206210 length 51
+    Stream: column 23 section DATA start: 1206261 length 782
+    Stream: column 24 section PRESENT start: 1207043 length 54
+    Stream: column 24 section DATA start: 1207097 length 783
+    Stream: column 25 section PRESENT start: 1207880 length 51
+    Stream: column 25 section DATA start: 1207931 length 5448
+    Stream: column 26 section PRESENT start: 1213379 length 53
+    Stream: column 26 section DATA start: 1213432 length 12144
+    Stream: column 27 section PRESENT start: 1225576 length 53
+    Stream: column 27 section DATA start: 1225629 length 24618
+    Stream: column 28 section PRESENT start: 1250247 length 52
+    Stream: column 28 section DATA start: 1250299 length 24681
+    Stream: column 29 section PRESENT start: 1274980 length 51
+    Stream: column 29 section DATA start: 1275031 length 9927
+    Stream: column 30 section PRESENT start: 1284958 length 53
+    Stream: column 30 section DATA start: 1285011 length 19755
+    Stream: column 31 section PRESENT start: 1304766 length 51
+    Stream: column 31 section DATA start: 1304817 length 11009
+    Stream: column 31 section LENGTH start: 1315826 length 3722
+    Stream: column 31 section DICTIONARY_DATA start: 1319548 length 65435
+    Stream: column 32 section PRESENT start: 1384983 length 54
+    Stream: column 32 section DATA start: 1385037 length 11006
+    Stream: column 32 section LENGTH start: 1396043 length 3739
+    Stream: column 32 section DICTIONARY_DATA start: 1399782 length 66022
+    Stream: column 33 section PRESENT start: 1465804 length 52
+    Stream: column 33 section DATA start: 1465856 length 8045
+    Stream: column 33 section SECONDARY start: 1473901 length 9555
+    Stream: column 34 section PRESENT start: 1483456 length 58
+    Stream: column 34 section DATA start: 1483514 length 8082
+    Stream: column 34 section SECONDARY start: 1491596 length 9590
+    Stream: column 35 section PRESENT start: 1501186 length 51
+    Stream: column 35 section DATA start: 1501237 length 782
+    Stream: column 36 section PRESENT start: 1502019 length 54
+    Stream: column 36 section DATA start: 1502073 length 783
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Encoding column 4: DIRECT_V2
+    Encoding column 5: DIRECT
+    Encoding column 6: DIRECT
+    Encoding column 7: DIRECT_V2
+    Encoding column 8: DIRECT_V2
+    Encoding column 9: DIRECT_V2
+    Encoding column 10: DIRECT_V2
+    Encoding column 11: DIRECT
+    Encoding column 12: DIRECT
+    Encoding column 13: DIRECT
+    Encoding column 14: DIRECT_V2
+    Encoding column 15: DIRECT_V2
+    Encoding column 16: DIRECT_V2
+    Encoding column 17: DIRECT
+    Encoding column 18: DIRECT
+    Encoding column 19: DICTIONARY_V2[6083]
+    Encoding column 20: DICTIONARY_V2[6081]
+    Encoding column 21: DIRECT_V2
+    Encoding column 22: DIRECT_V2
+    Encoding column 23: DIRECT
+    Encoding column 24: DIRECT
+    Encoding column 25: DIRECT
+    Encoding column 26: DIRECT_V2
+    Encoding column 27: DIRECT_V2
+    Encoding column 28: DIRECT_V2
+    Encoding column 29: DIRECT
+    Encoding column 30: DIRECT
+    Encoding column 31: DICTIONARY_V2[6083]
+    Encoding column 32: DICTIONARY_V2[6081]
+    Encoding column 33: DIRECT_V2
+    Encoding column 34: DIRECT_V2
+    Encoding column 35: DIRECT
+    Encoding column 36: DIRECT
+    Row group indices for column 0:
+      Entry 0: count: 10000 hasNull: false positions: 
+      Entry 1: count: 2288 hasNull: false positions: 
+    Row group indices for column 1:
+      Entry 0: count: 7909 hasNull: true min: -64 max: 62 sum: -50203 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64 max: 62 sum: 10347 positions: 0,182,99,0,0,5937,2
+    Row group indices for column 2:
+      Entry 0: count: 7924 hasNull: true min: -16379 max: 16376 sum: 9298530 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309 max: 16331 sum: -1862540 positions: 0,126,96,0,0,15334,272
+    Row group indices for column 3:
+      Entry 0: count: 7139 hasNull: true min: -1073051226 max: 1073680599 sum: 1417841516466 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: -1073279343 max: 1072872630 sum: 20209347319 positions: 0,128,98,0,0,28584,0
+    Row group indices for column 4:
+      Entry 0: count: 6889 hasNull: true min: -2147311592 max: 2144325818 sum: -24788202148 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: -2144905793 max: 2145498388 sum: -1673671826261 positions: 0,168,7,0,0,26534,262
+    Row group indices for column 5:
+      Entry 0: count: 7909 hasNull: true min: -64.0 max: 79.5530014038086 sum: -49823.35599219799 positions: 0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64.0 max: 62.0 sum: 10343.719999313354 positions: 0,182,99,0,0,31636
+    Row group indices for column 6:
+      Entry 0: count: 7924 hasNull: true min: -16379.0 max: 9763215.5639 sum: 4.8325701237600006E7 positions: 0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309.0 max: 9763215.5639 sum: 7897951.792899999 positions: 0,126,96,0,0,63392
+    Row group indices for column 7:
+      Entry 0: count: 10000 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 360000 positions: 0,0,0,0,0
+      Entry 1: count: 2288 hasNull: false min: 002d8ccb-a094-4d10-b283-999770cf8488 max: ffacef94-41da-4230-807a-509bbf50b057 sum: 82368 positions: 153190,97856,0,9766,272
+    Row group indices for column 8:
+      Entry 0: count: 10000 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 720000 positions: 0,0,0,0,0
+      Entry 1: count: 2288 hasNull: false min: 00124556-8383-44c4-a28b-7a413de74ccc4137606f-2cf7-43fb-beff-b6d374fd15ec max: ffde3bce-bb56-4fa9-81d7-146ca2eab946225c18e0-0002-4d07-9853-12c92c0f5637 sum: 164736 positions: 306445,195712,0,9766,272
+    Row group indices for column 9:
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 13:59:43.64 max: 1969-12-31 14:00:30.808 positions: 0,182,100,0,0,30619,258,0,15332,258
+    Row group indices for column 10:
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,126,97,0,0,30619,273,0,15334,272
+    Row group indices for column 11:
+      Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
+    Row group indices for column 12:
+      Entry 0: count: 6889 hasNull: true true: 3402 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true true: 581 positions: 0,168,8,0,0,520,97,1
+    Row group indices for column 13:
+      Entry 0: count: 7909 hasNull: true min: -64 max: 62 sum: -50203 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64 max: 62 sum: 10347 positions: 0,182,99,0,0,5937,2
+    Row group indices for column 14:
+      Entry 0: count: 7924 hasNull: true min: -16379 max: 16376 sum: 9298530 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309 max: 16331 sum: -1862540 positions: 0,126,96,0,0,15334,272
+    Row group indices for column 15:
+      Entry 0: count: 7139 hasNull: true min: -1073051226 max: 1073680599 sum: 1417841516466 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: -1073279343 max: 1072872630 sum: 20209347319 positions: 0,128,98,0,0,28584,0
+    Row group indices for column 16:
+      Entry 0: count: 6889 hasNull: true min: -2147311592 max: 2144325818 sum: -24788202148 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: -2144905793 max: 2145498388 sum: -1673671826261 positions: 0,168,7,0,0,26534,262
+    Row group indices for column 17:
+      Entry 0: count: 7909 hasNull: true min: -64.0 max: 79.5530014038086 sum: -49823.35599219799 positions: 0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64.0 max: 62.0 sum: 10343.719999313354 positions: 0,182,99,0,0,31636
+    Row group indices for column 18:
+      Entry 0: count: 7924 hasNull: true min: -16379.0 max: 9763215.5639 sum: 4.8325701237600006E7 positions: 0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309.0 max: 9763215.5639 sum: 7897951.792899999 positions: 0,126,96,0,0,63392
+    Row group indices for column 19:
+      Entry 0: count: 7140 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yxN0212hM17E8J8bJj8D7b sum: 99028 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: 006bb3K max: yy2GiGM sum: 28853 positions: 0,126,98,0,0,14308,0
+    Row group indices for column 20:
+      Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,13280,262
+    Row group indices for column 21:
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 13:59:43.64 max: 1969-12-31 14:00:30.808 positions: 0,182,100,0,0,30619,258,0,15332,258
+    Row group indices for column 22:
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,126,97,0,0,30619,273,0,15334,272
+    Row group indices for column 23:
+      Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
+    Row group indices for column 24:
+      Entry 0: count: 6889 hasNull: true true: 3402 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true true: 581 positions: 0,168,8,0,0,520,97,1
+    Row group indices for column 25:
+      Entry 0: count: 7909 hasNull: true min: -64 max: 62 sum: -50203 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64 max: 62 sum: 10347 positions: 0,182,99,0,0,5937,2
+    Row group indices for column 26:
+      Entry 0: count: 7924 hasNull: true min: -16379 max: 16376 sum: 9298530 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309 max: 16331 sum: -1862540 positions: 0,126,96,0,0,15334,272
+    Row group indices for column 27:
+      Entry 0: count: 7139 hasNull: true min: -1073051226 max: 1073680599 sum: 1417841516466 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: -1073279343 max: 1072872630 sum: 20209347319 positions: 0,128,98,0,0,28584,0
+    Row group indices for column 28:
+      Entry 0: count: 6889 hasNull: true min: -2147311592 max: 2144325818 sum: -24788202148 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: -2144905793 max: 2145498388 sum: -1673671826261 positions: 0,168,7,0,0,26534,262
+    Row group indices for column 29:
+      Entry 0: count: 7909 hasNull: true min: -64.0 max: 79.5530014038086 sum: -49823.35599219799 positions: 0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64.0 max: 62.0 sum: 10343.719999313354 positions: 0,182,99,0,0,31636
+    Row group indices for column 30:
+      Entry 0: count: 7924 hasNull: true min: -16379.0 max: 9763215.5639 sum: 4.8325701237600006E7 positions: 0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309.0 max: 9763215.5639 sum: 7897951.792899999 positions: 0,126,96,0,0,63392
+    Row group indices for column 31:
+      Entry 0: count: 7140 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yxN0212hM17E8J8bJj8D7b sum: 99028 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: 006bb3K max: yy2GiGM sum: 28853 positions: 0,126,98,0,0,14308,0
+    Row group indices for column 32:
+      Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,13280,262
+    Row group indices for column 33:
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 13:59:43.64 max: 1969-12-31 14:00:30.808 positions: 0,182,100,0,0,30619,258,0,15332,258
+    Row group indices for column 34:
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,126,97,0,0,30619,273,0,15334,272
+    Row group indices for column 35:
+      Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
+    Row group indices for column 36:
+      Entry 0: count: 6889 hasNull: true true: 3402 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true true: 581 positions: 0,168,8,0,0,520,97,1
+  Stripe: offset: 1503357 data: 1500017 rows: 12288 tail: 501 index: 2836
+    Stream: column 0 section ROW_INDEX start: 1503357 length 21
+    Stream: column 1 section ROW_INDEX start: 1503378 length 53
+    Stream: column 2 section ROW_INDEX start: 1503431 length 67
+    Stream: column 3 section ROW_INDEX start: 1503498 length 81
+    Stream: column 4 section ROW_INDEX start: 1503579 length 83
+    Stream: column 5 section ROW_INDEX start: 1503662 length 77
+    Stream: column 6 section ROW_INDEX start: 1503739 length 77
+    Stream: column 7 section ROW_INDEX start: 1503816 length 176
+    Stream: column 8 section ROW_INDEX start: 1503992 length 267
+    Stream: column 9 section ROW_INDEX start: 1504259 length 63
+    Stream: column 10 section ROW_INDEX start: 1504322 length 57
+    Stream: column 11 section ROW_INDEX start: 1504379 length 47
+    Stream: column 12 section ROW_INDEX start: 1504426 length 47
+    Stream: column 13 section ROW_INDEX start: 1504473 length 53
+    Stream: column 14 section ROW_INDEX start: 1504526 length 67
+    Stream: column 15 section ROW_INDEX start: 1504593 length 81
+    Stream: column 16 section ROW_INDEX start: 1504674 length 83
+    Stream: column 17 section ROW_INDEX start: 1504757 length 77
+    Stream: column 18 section ROW_INDEX start: 1504834 length 77
+    Stream: column 19 section ROW_INDEX start: 1504911 length 115
+    Stream: column 20 section ROW_INDEX start: 1505026 length 93
+    Stream: column 21 section ROW_INDEX start: 1505119 length 63
+    Stream: column 22 section ROW_INDEX start: 1505182 length 57
+    Stream: column 23 section ROW_INDEX start: 1505239 length 47
+    Stream: column 24 section ROW_INDEX start: 1505286 length 47
+    Stream: column 25 section ROW_INDEX start: 1505333 length 53
+    Stream: column 26 section ROW_INDEX start: 1505386 length 67
+    Stream: column 27 section ROW_INDEX start: 1505453 length 81
+    Stream: column 28 section ROW_INDEX start: 1505534 length 83
+    Stream: column 29 section ROW_INDEX start: 1505617 length 77
+    Stream: column 30 section ROW_INDEX start: 1505694 length 77
+    Stream: column 31 section ROW_INDEX start: 1505771 length 115
+    Stream: column 32 section ROW_INDEX start: 1505886 length 93
+    Stream: column 33 section ROW_INDEX start: 1505979 length 63
+    Stream: column 34 section ROW_INDEX start: 1506042 length 57
+    Stream: column 35 section ROW_INDEX start: 1506099 length 47
+    Stream: column 36 section ROW_INDEX start: 1506146 length 47
+    Stream: column 1 section PRESENT start: 1506193 length 51
+    Stream: column 1 section DATA start: 1506244 length 5448
+    Stream: column 2 section PRESENT start: 1511692 length 53
+    Stream: column 2 section DATA start: 1511745 length 12144
+    Stream: column 3 section PRESENT start: 1523889 length 53
+    Stream: column 3 section DATA start: 1523942 length 24618
+    Stream: column 4 section PRESENT start: 1548560 length 52
+    Stream: column 4 section DATA start: 1548612 length 24681
+    Stream: column 5 section PRESENT start: 1573293 length 51
+    Stream: column 5 section DATA start: 1573344 length 9927
+    Stream: column 6 section PRESENT start: 1583271 length 53
+    Stream: column 6 section DATA start: 1583324 length 19755
+    Stream: column 7 section DATA start: 1603079 length 258570
+    Stream: column 7 section LENGTH start: 1861649 length 108
+    Stream: column 8 section DATA start: 1861757 length 517341
+    Stream: column 8 section LENGTH start: 2379098 length 108
+    Stream: column 9 section PRESENT start: 2379206 length 52
+    Stream: column 9 section DATA start: 2379258 length 8045
+    Stream: column 9 section SECONDARY start: 2387303 length 9555
+    Stream: column 10 section PRESENT start: 2396858 length 58
+    Stream: column 10 section DATA start: 2396916 length 8082
+    Stream: column 10 section SECONDARY start: 2404998 length 9590
+    Stream: column 11 section PRESENT start: 2414588 length 51
+    Stream: column 11 section DATA start: 2414639 length 782
+    Stream: column 12 section PRESENT start: 2415421 length 54
+    Stream: column 12 section DATA start: 2415475 length 783
+    Stream: column 13 section PRESENT start: 2416258 length 51
+    Stream: column 13 section DATA start: 2416309 length 5448
+    Stream: column 14 section PRESENT start: 2421757 length 53
+    Stream: column 14 section DATA start: 2421810 length 12144
+    Stream: column 15 section PRESENT start: 2433954 length 53
+    Stream: column 15 section DATA start: 2434007 length 24618
+    Stream: column 16 section PRESENT start: 2458625 length 52
+    Stream: column 16 section DATA start: 2458677 length 24681
+    Stream: column 17 section PRESENT start: 2483358 length 51
+    Stream: column 17 section DATA start: 2483409 length 9927
+    Stream: column 18 section PRESENT start: 2493336 length 53
+    Stream: column 18 section DATA start: 2493389 length 19755
+    Stream: column 19 section PRESENT start: 2513144 length 51
+    Stream: column 19 section DATA start: 2513195 length 11009
+    Stream: column 19 section LENGTH start: 2524204 length 3722
+    Stream: column 19 section DICTIONARY_DATA start: 2527926 length 65435
+    Stream: column 20 section PRESENT start: 2593361 length 54
+    Stream: column 20 section DATA start: 2593415 length 11006
+    Stream: column 20 section LENGTH start: 2604421 length 3739
+    Stream: column 20 section DICTIONARY_DATA start: 2608160 length 66022
+    Stream: column 21 section PRESENT start: 2674182 length 52
+    Stream: column 21 section DATA start: 2674234 length 8045
+    Stream: column 21 section SECONDARY start: 2682279 length 9555
+    Stream: column 22 section PRESENT start: 2691834 length 58
+    Stream: column 22 section DATA start: 2691892 length 8082
+    Stream: column 22 section SECONDARY start: 2699974 length 9590
+    Stream: column 23 section PRESENT start: 2709564 length 51
+    Stream: column 23 section DATA start: 2709615 length 782
+    Stream: column 24 section PRESENT start: 2710397 length 54
+    Stream: column 24 section DATA start: 2710451 length 783
+    Stream: column 25 section PRESENT start: 2711234 length 51
+    Stream: column 25 section DATA start: 2711285 length 5448
+    Stream: column 26 section PRESENT start: 2716733 length 53
+    Stream: column 26 section DATA start: 2716786 length 12144
+    Stream: column 27 section PRESENT start: 2728930 length 53
+    Stream: column 27 section DATA start: 2728983 length 24618
+    Stream: column 28 section PRESENT start: 2753601 length 52
+    Stream: column 28 section DATA start: 2753653 length 24681
+    Stream: column 29 section PRESENT start: 2778334 length 51
+    Stream: column 29 section DATA start: 2778385 length 9927
+    Stream: column 30 section PRESENT start: 2788312 length 53
+    Stream: column 30 section DATA start: 2788365 length 19755
+    Stream: column 31 section PRESENT start: 2808120 length 51
+    Stream: column 31 section DATA start: 2808171 length 11009
+    Stream: column 31 section LENGTH start: 2819180 length 3722
+    Stream: column 31 section DICTIONARY_DATA start: 2822902 length 65435
+    Stream: column 32 section PRESENT start: 2888337 length 54
+    Stream: column 32 section DATA start: 2888391 length 11006
+    Stream: column 32 section LENGTH start: 2899397 length 3739
+    Stream: column 32 section DICTIONARY_DATA start: 2903136 length 66022
+    Stream: column 33 section PRESENT start: 2969158 length 52
+    Stream: column 33 section DATA start: 2969210 length 8045
+    Stream: column 33 section SECONDARY start: 2977255 length 9555
+    Stream: column 34 section PRESENT start: 2986810 length 58
+    Stream: column 34 section DATA start: 2986868 length 8082
+    Stream: column 34 section SECONDARY start: 2994950 length 9590
+    Stream: column 35 section PRESENT start: 3004540 length 51
+    Stream: column 35 section DATA start: 3004591 length 782
+    Stream: column 36 section PRESENT start: 3005373 length 54
+    Stream: column 36 section DATA start: 3005427 length 783
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Encoding column 4: DIRECT_V2
+    Encoding column 5: DIRECT
+    Encoding column 6: DIRECT
+    Encoding column 7: DIRECT_V2
+    Encoding column 8: DIRECT_V2
+    Encoding column 9: DIRECT_V2
+    Encoding column 10: DIRECT_V2
+    Encoding column 11: DIRECT
+    Encoding column 12: DIRECT
+    Encoding column 13: DIRECT
+    Encoding column 14: DIRECT_V2
+    Encoding column 15: DIRECT_V2
+    Encoding column 16: DIRECT_V2
+    Encoding column 17: DIRECT
+    Encoding column 18: DIRECT
+    Encoding column 19: DICTIONARY_V2[6083]
+    Encoding column 20: DICTIONARY_V2[6081]
+    Encoding column 21: DIRECT_V2
+    Encoding column 22: DIRECT_V2
+    Encoding column 23: DIRECT
+    Encoding column 24: DIRECT
+    Encoding column 25: DIRECT
+    Encoding column 26: DIRECT_V2
+    Encoding column 27: DIRECT_V2
+    Encoding column 28: DIRECT_V2
+    Encoding column 29: DIRECT
+    Encoding column 30: DIRECT
+    Encoding column 31: DICTIONARY_V2[6083]
+    Encoding column 32: DICTIONARY_V2[6081]
+    Encoding column 33: DIRECT_V2
+    Encoding column 34: DIRECT_V2
+    Encoding column 35: DIRECT
+    Encoding column 36: DIRECT
+    Row group indices for column 0:
+      Entry 0: count: 10000 hasNull: false positions: 
+      Entry 1: count: 2288 hasNull: false positions: 
+    Row group indices for column 1:
+      Entry 0: count: 7909 hasNull: true min: -64 max: 62 sum: -50203 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64 max: 62 sum: 10347 positions: 0,182,99,0,0,5937,2
+    Row group indices for column 2:
+      Entry 0: count: 7924 hasNull: true min: -16379 max: 16376 sum: 9298530 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309 max: 16331 sum: -1862540 positions: 0,126,96,0,0,15334,272
+    Row group indices for column 3:
+      Entry 0: count: 7139 hasNull: true min: -1073051226 max: 1073680599 sum: 1417841516466 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: -1073279343 max: 1072872630 sum: 20209347319 positions: 0,128,98,0,0,28584,0
+    Row group indices for column 4:
+      Entry 0: count: 6889 hasNull: true min: -2147311592 max: 2144325818 sum: -24788202148 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: -2144905793 max: 2145498388 sum: -1673671826261 positions: 0,168,7,0,0,26534,262
+    Row group indices for column 5:
+      Entry 0: count: 7909 hasNull: true min: -64.0 max: 79.5530014038086 sum: -49823.35599219799 positions: 0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64.0 max: 62.0 sum: 10343.719999313354 positions: 0,182,99,0,0,31636
+    Row group indices for column 6:
+      Entry 0: count: 7924 hasNull: true min: -16379.0 max: 9763215.5639 sum: 4.8325701237600006E7 positions: 0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309.0 max: 9763215.5639 sum: 7897951.792899999 positions: 0,126,96,0,0,63392
+    Row group indices for column 7:
+      Entry 0: count: 10000 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 360000 positions: 0,0,0,0,0
+      Entry 1: count: 2288 hasNull: false min: 002d8ccb-a094-4d10-b283-999770cf8488 max: ffacef94-41da-4230-807a-509bbf50b057 sum: 82368 positions: 153190,97856,0,9766,272
+    Row group indices for column 8:
+      Entry 0: count: 10000 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 720000 positions: 0,0,0,0,0
+      Entry 1: count: 2288 hasNull: false min: 00124556-8383-44c4-a28b-7a413de74ccc4137606f-2cf7-43fb-beff-b6d374fd15ec max: ffde3bce-bb56-4fa9-81d7-146ca2eab946225c18e0-0002-4d07-9853-12c92c0f5637 sum: 164736 positions: 306445,195712,0,9766,272
+    Row group indices for column 9:
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 13:59:43.64 max: 1969-12-31 14:00:30.808 positions: 0,182,100,0,0,30619,258,0,15332,258
+    Row group indices for column 10:
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,126,97,0,0,30619,273,0,15334,272
+    Row group indices for column 11:
+      Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
+    Row group indices for column 12:
+      Entry 0: count: 6889 hasNull: true true: 3402 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true true: 581 positions: 0,168,8,0,0,520,97,1
+    Row group indices for column 13:
+      Entry 0: count: 7909 hasNull: true min: -64 max: 62 sum: -50203 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64 max: 62 sum: 10347 positions: 0,182,99,0,0,5937,2
+    Row group indices for column 14:
+      Entry 0: count: 7924 hasNull: true min: -16379 max: 16376 sum: 9298530 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309 max: 16331 sum: -1862540 positions: 0,126,96,0,0,15334,272
+    Row group indices for column 15:
+      Entry 0: count: 7139 hasNull: true min: -1073051226 max: 1073680599 sum: 1417841516466 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: -1073279343 max: 1072872630 sum: 20209347319 positions: 0,128,98,0,0,28584,0
+    Row group indices for column 16:
+      Entry 0: count: 6889 hasNull: true min: -2147311592 max: 2144325818 sum: -24788202148 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: -2144905793 max: 2145498388 sum: -1673671826261 positions: 0,168,7,0,0,26534,262
+    Row group indices for column 17:
+      Entry 0: count: 7909 hasNull: true min: -64.0 max: 79.5530014038086 sum: -49823.35599219799 positions: 0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64.0 max: 62.0 sum: 10343.719999313354 positions: 0,182,99,0,0,31636
+    Row group indices for column 18:
+      Entry 0: count: 7924 hasNull: true min: -16379.0 max: 9763215.5639 sum: 4.8325701237600006E7 positions: 0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309.0 max: 9763215.5639 sum: 7897951.792899999 positions: 0,126,96,0,0,63392
+    Row group indices for column 19:
+      Entry 0: count: 7140 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yxN0212hM17E8J8bJj8D7b sum: 99028 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: 006bb3K max: yy2GiGM sum: 28853 positions: 0,126,98,0,0,14308,0
+    Row group indices for column 20:
+      Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,13280,262
+    Row group indices for column 21:
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 13:59:43.64 max: 1969-12-31 14:00:30.808 positions: 0,182,100,0,0,30619,258,0,15332,258
+    Row group indices for column 22:
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,126,97,0,0,30619,273,0,15334,272
+    Row group indices for column 23:
+      Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
+    Row group indices for column 24:
+      Entry 0: count: 6889 hasNull: true true: 3402 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true true: 581 positions: 0,168,8,0,0,520,97,1
+    Row group indices for column 25:
+      Entry 0: count: 7909 hasNull: true min: -64 max: 62 sum: -50203 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64 max: 62 sum: 10347 positions: 0,182,99,0,0,5937,2
+    Row group indices for column 26:
+      Entry 0: count: 7924 hasNull: true min: -16379 max: 16376 sum: 9298530 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309 max: 16331 sum: -1862540 positions: 0,126,96,0,0,15334,272
+    Row group indices for column 27:
+      Entry 0: count: 7139 hasNull: true min: -1073051226 max: 1073680599 sum: 1417841516466 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: -1073279343 max: 1072872630 sum: 20209347319 positions: 0,128,98,0,0,28584,0
+    Row group indices for column 28:
+      Entry 0: count: 6889 hasNull: true min: -2147311592 max: 2144325818 sum: -24788202148 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: -2144905793 max: 2145498388 sum: -1673671826261 positions: 0,168,7,0,0,26534,262
+    Row group indices for column 29:
+      Entry 0: count: 7909 hasNull: true min: -64.0 max: 79.5530014038086 sum: -49823.35599219799 positions: 0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: -64.0 max: 62.0 sum: 10343.719999313354 positions: 0,182,99,0,0,31636
+    Row group indices for column 30:
+      Entry 0: count: 7924 hasNull: true min: -16379.0 max: 9763215.5639 sum: 4.8325701237600006E7 positions: 0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: -16309.0 max: 9763215.5639 sum: 7897951.792899999 positions: 0,126,96,0,0,63392
+    Row group indices for column 31:
+      Entry 0: count: 7140 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yxN0212hM17E8J8bJj8D7b sum: 99028 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true min: 006bb3K max: yy2GiGM sum: 28853 positions: 0,126,98,0,0,14308,0
+    Row group indices for column 32:
+      Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,13280,262
+    Row group indices for column 33:
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 13:59:43.64 max: 1969-12-31 14:00:30.808 positions: 0,182,100,0,0,30619,258,0,15332,258
+    Row group indices for column 34:
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 13:59:30.929 max: 1969-12-31 14:00:30.808 positions: 0,126,97,0,0,30619,273,0,15334,272
+    Row group indices for column 35:
+      Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
+    Row group indices for column 36:
+      Entry 0: count: 6889 hasNull: true true: 3402 positions: 0,0,0,0,0,0,0,0
+      Entry 1: count: 2284 hasNull: true true: 581 positions: 0,168,8,0,0,520,97,1
+
+File length: 3007981 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+
+-- END ORC FILE DUMP --
+-50	-13326	528534767	NULL	-50.0	-13326.0	18f2de7d-0c69-4052-9c4b-64f196d6d589	9edba9e8-1f91-47e7-b31f-451eeb5feb7781ba3a3e-c22e-4412-91a2-86ad05ae1ca8	1969-12-31 15:59:46.674	1969-12-31 16:00:07.875	true	NULL	-50	-13326	528534767	NULL	-50.0	-13326.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:46.674	1969-12-31 16:00:07.875	true	NULL	-50	-13326	528534767	NULL	-50.0	-13326.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:46.674	1969-12-31 16:00:07.875	true	NULL