You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2015/08/24 06:44:34 UTC

hive git commit: HIVE-11506 : Casting varchar/char type to string cannot be vectorized (Navis via Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master e2d148ba8 -> 2de6fda0f


HIVE-11506 : Casting varchar/char type to string cannot be vectorized (Navis via Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: 2de6fda0f81072a6ad1af103f2ddec49b73f4eab
Parents: e2d148b
Author: Navis Ryu <na...@apache.org>
Authored: Mon Aug 10 17:36:00 2015 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Sun Aug 23 21:32:00 2015 -0700

----------------------------------------------------------------------
 .../ql/exec/vector/VectorizationContext.java    |  7 +-
 .../expressions/CastStringGroupToString.java    | 40 ++++++++
 .../queries/clientpositive/vectorized_casts.q   |  6 ++
 .../tez/vector_char_mapjoin1.q.out              |  1 +
 .../tez/vector_varchar_mapjoin1.q.out           |  1 +
 .../clientpositive/tez/vectorized_casts.q.out   | 99 +++++++++++++-------
 .../clientpositive/vector_char_mapjoin1.q.out   |  1 +
 .../vector_varchar_mapjoin1.q.out               |  1 +
 .../clientpositive/vectorized_casts.q.out       | 66 +++++++------
 9 files changed, 155 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 5b702bb..34f9329 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -50,7 +50,6 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.InputExpressionType;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.Mode;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
-import org.apache.hadoop.hive.ql.exec.vector.AggregateDefinition;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFAvgDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFCount;
@@ -1023,9 +1022,7 @@ public class VectorizationContext {
     VectorExpressionDescriptor.Descriptor descriptor = builder.build();
     Class<?> vclass = this.vMap.getVectorExpressionClass(udfClass, descriptor);
     if (vclass == null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("No vector udf found for "+udfClass.getSimpleName() + ", descriptor: "+descriptor);
-      }
+      LOG.info("No vector udf found for " + udfClass.getSimpleName() + ", descriptor: " + descriptor);
       return null;
     }
     Mode childrenMode = getChildrenMode(mode, udfClass);
@@ -1536,6 +1533,8 @@ public class VectorizationContext {
       return createVectorExpression(CastDecimalToString.class, childExpr, Mode.PROJECTION, returnType);
     } else if (isDateFamily(inputType)) {
       return createVectorExpression(CastDateToString.class, childExpr, Mode.PROJECTION, returnType);
+    } else if (isStringFamily(inputType)) {
+      return createVectorExpression(CastStringGroupToString.class, childExpr, Mode.PROJECTION, returnType);
     }
     /* The string type is deliberately omitted -- the planner removes string to string casts.
      * Timestamp, float, and double types are handled by the legacy code path. See isLegacyPathUDF.

http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringGroupToString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringGroupToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringGroupToString.java
new file mode 100644
index 0000000..bbc770c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringGroupToString.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+// cast string group to string (varchar to string, etc.)
+public class CastStringGroupToString extends StringUnaryUDFDirect {
+
+  private static final long serialVersionUID = 1L;
+
+  public CastStringGroupToString() {
+    super();
+  }
+
+  public CastStringGroupToString(int inputColumn, int outputColumn) {
+    super(inputColumn, outputColumn);
+  }
+
+  @Override
+  protected void func(BytesColumnVector outV, byte[][] vector, int[] start, int[] length, int i) {
+    outV.setVal(i, vector[i], start[i], length[i]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/test/queries/clientpositive/vectorized_casts.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorized_casts.q b/ql/src/test/queries/clientpositive/vectorized_casts.q
index ade3645..8e0bdbc 100644
--- a/ql/src/test/queries/clientpositive/vectorized_casts.q
+++ b/ql/src/test/queries/clientpositive/vectorized_casts.q
@@ -1,8 +1,10 @@
 set hive.explain.user=false;
+set hive.fetch.task.conversion=none;
 SET hive.vectorized.execution.enabled = true;
 
 -- SORT_QUERY_RESULTS
 
+-- Currently, vectorization is not supported in fetch task (hive.fetch.task.conversion=none)
 -- Test type casting in vectorized mode to verify end-to-end functionality.
 
 explain 
@@ -68,6 +70,8 @@ select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -140,6 +144,8 @@ select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)

http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out b/ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out
index f495f95..cdb24cc 100644
--- a/ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_char_mapjoin1.q.out
@@ -366,6 +366,7 @@ STAGE PLANS:
                           sort order: +
                           Statistics: Num rows: 2 Data size: 215 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: string)
+            Execution mode: vectorized
         Map 3 
             Map Operator Tree:
                 TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/test/results/clientpositive/tez/vector_varchar_mapjoin1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_varchar_mapjoin1.q.out b/ql/src/test/results/clientpositive/tez/vector_varchar_mapjoin1.q.out
index ae5be44..b453b0e 100644
--- a/ql/src/test/results/clientpositive/tez/vector_varchar_mapjoin1.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_varchar_mapjoin1.q.out
@@ -354,6 +354,7 @@ STAGE PLANS:
                           sort order: +
                           Statistics: Num rows: 2 Data size: 200 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col1 (type: varchar(10)), _col2 (type: int), _col3 (type: string)
+            Execution mode: vectorized
         Map 3 
             Map Operator Tree:
                 TableScan

http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out b/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
index 1dd07b1..19c41ed 100644
--- a/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
@@ -1,5 +1,6 @@
 PREHOOK: query: -- SORT_QUERY_RESULTS
 
+-- Currently, vectorization is not supported in fetch task (hive.fetch.task.conversion=none)
 -- Test type casting in vectorized mode to verify end-to-end functionality.
 
 explain 
@@ -65,6 +66,8 @@ select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -76,6 +79,7 @@ where cbigint % 250 = 0
 PREHOOK: type: QUERY
 POSTHOOK: query: -- SORT_QUERY_RESULTS
 
+-- Currently, vectorization is not supported in fetch task (hive.fetch.task.conversion=none)
 -- Test type casting in vectorized mode to verify end-to-end functionality.
 
 explain 
@@ -141,6 +145,8 @@ select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -151,21 +157,40 @@ from alltypesorc
 where cbigint % 250 = 0
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
-  Stage-0 is a root stage
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint % 250) = 0) (type: boolean)
+                    Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), UDFToBoolean((cbigint * 0)) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (type: int), UDFToInteger(cbigint) (type: int), UDFToInteger(cfloat) (type: int), UDFToInteger(cdouble) (type: int), UDFToInteger(cboolean1) (type: int), UDFToInteger(ctimestamp1) (type: int), UDFToInteger(cstring1) (type: int), UDFToInteger(substr(cstring1, 1, 1)) (type: int), UDFToByte(cfloat) (type: tinyint), UDFToShort(cfloat) (type: smallint), UDFToLong(cfloat) (type: bigint), UDFToDouble(ctinyint) (type: double), UDFToDouble(csmallint) (type: double), UDFToDouble(cint) (type: double), 
 UDFToDouble(cbigint) (type: double), UDFToDouble(cfloat) (type: double), cdouble (type: double), UDFToDouble(cboolean1) (type: double), UDFToDouble(ctimestamp1) (type: double), UDFToDouble(cstring1) (type: double), UDFToDouble(substr(cstring1, 1, 1)) (type: double), UDFToFloat(cint) (type: float), UDFToFloat(cdouble) (type: float), CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp), CAST( substr(cstring1, 1, 1) AS TIMESTAMP) (type: timestamp), UDFToString(ctinyint) (type: string), UDFToString(csmallint) (type: string), UDFToString(cint) (type: string), UDFToString(cbigint) (type: 
 string), UDFToString(cfloat) (type: string), UDFToString(cdouble) (type: string), UDFToString(cboolean1) (type: string), UDFToString((cbigint * 0)) (type: string), UDFToString(ctimestamp1) (type: string), cstring1 (type: string), UDFToString(CAST( cstring1 AS CHAR(10)) (type: string), UDFToString(CAST( cstring1 AS varchar(10))) (type: string), UDFToFloat(UDFToInteger(cfloat)) (type: float), UDFToDouble((cint * 2)) (type: double), UDFToString(sin(cfloat)) (type: string), (UDFToDouble(UDFToFloat(cint)) + UDFToDouble(cboolean1)) (type: double)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col57, _col58, _col59, _col60, _col61
+                      Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized
+
   Stage: Stage-0
     Fetch Operator
       limit: -1
       Processor Tree:
-        TableScan
-          alias: alltypesorc
-          Filter Operator
-            predicate: ((cbigint % 250) = 0) (type: boolean)
-            Select Operator
-              expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), UDFToBoolean((cbigint * 0)) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (type: int), UDFToInteger(cbigint) (type: int), UDFToInteger(cfloat) (type: int), UDFToInteger(cdouble) (type: int), UDFToInteger(cboolean1) (type: int), UDFToInteger(ctimestamp1) (type: int), UDFToInteger(cstring1) (type: int), UDFToInteger(substr(cstring1, 1, 1)) (type: int), UDFToByte(cfloat) (type: tinyint), UDFToShort(cfloat) (type: smallint), UDFToLong(cfloat) (type: bigint), UDFToDouble(ctinyint) (type: double), UDFToDouble(csmallint) (type: double), UDFToDouble(cint) (type: double), UDFToDou
 ble(cbigint) (type: double), UDFToDouble(cfloat) (type: double), cdouble (type: double), UDFToDouble(cboolean1) (type: double), UDFToDouble(ctimestamp1) (type: double), UDFToDouble(cstring1) (type: double), UDFToDouble(substr(cstring1, 1, 1)) (type: double), UDFToFloat(cint) (type: float), UDFToFloat(cdouble) (type: float), CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp), CAST( substr(cstring1, 1, 1) AS TIMESTAMP) (type: timestamp), UDFToString(ctinyint) (type: string), UDFToString(csmallint) (type: string), UDFToString(cint) (type: string), UDFToString(cbigint) (type: string),
  UDFToString(cfloat) (type: string), UDFToString(cdouble) (type: string), UDFToString(cboolean1) (type: string), UDFToString((cbigint * 0)) (type: string), UDFToString(ctimestamp1) (type: string), cstring1 (type: string), UDFToFloat(UDFToInteger(cfloat)) (type: float), UDFToDouble((cint * 2)) (type: double), UDFToString(sin(cfloat)) (type: string), (UDFToDouble(UDFToFloat(cint)) + UDFToDouble(cboolean1)) (type: double)
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col57, _col58, _col59
-              ListSink
+        ListSink
 
 PREHOOK: query: select 
 -- to boolean
@@ -229,6 +254,8 @@ PREHOOK: query: select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -302,6 +329,8 @@ POSTHOOK: query: select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -313,29 +342,29 @@ where cbigint % 250 = 0
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-true	NULL	true	true	true	NULL	false	false	true	true	-51	NULL	773600971	1053923250	-51	NULL	0	8	NULL	2	-51	-51	-51	-51.0	NULL	7.73600971E8	1.05392325E9	-51.0	NULL	0.0	8.451	NULL	2.0	7.7360096E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 14:53:20.971	1970-01-12 20:45:23.25	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	773600971	1053923250	-51.0	NULL	FALSE	0	1969-12-31 16:00:08.451	2yK4Bx76O	-51.0	1.547201942E9	-0.6702291758433747	7.7360096E8
-true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-102936434	-1312782750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-1.02936434E8	-1.31278275E9	8.0	NULL	0.0	15.892	NULL	NULL	-1.02936432E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-30 11:24:23.566	1969-12-16 11:20:17.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-102936434	-1312782750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	eJROSNhugc3kQR7Pb	8.0	-2.05872868E8	0.9893582466233818	-1.02936432E8
-true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-661621138	-931392750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-6.61621138E8	-9.3139275E8	8.0	NULL	0.0	15.892	NULL	NULL	-6.6162112E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-24 00:12:58.862	1969-12-20 21:16:47.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-661621138	-931392750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	L15l8i5k558tBcDV20	8.0	-1.323242276E9	0.9893582466233818	-6.6162112E8
-true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-669632311	1588591250	8	NULL	0	15	NULL	3	8	8	8	8.0	NULL	-6.69632311E8	1.58859125E9	8.0	NULL	0.0	15.892	NULL	3.0	-6.6963232E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-23 21:59:27.689	1970-01-19 01:16:31.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-669632311	1588591250	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	3r3sDvfUkG0yTP3LnX5mNQRr	8.0	-1.339264622E9	0.9893582466233818	-6.6963232E8
-true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	805179664	868161500	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	8.05179664E8	8.681615E8	8.0	NULL	0.0	15.892	NULL	NULL	8.0517965E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-09 23:39:39.664	1970-01-10 17:09:21.5	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	805179664	868161500	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	e005B5q	8.0	1.610359328E9	0.9893582466233818	8.05179648E8
-true	NULL	true	true	true	NULL	true	false	true	true	-51	NULL	747553882	-1930467250	-51	NULL	1	8	NULL	NULL	-51	-51	-51	-51.0	NULL	7.47553882E8	-1.93046725E9	-51.0	NULL	1.0	8.451	NULL	NULL	7.4755386E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 07:39:13.882	1969-12-09 07:45:32.75	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	747553882	-1930467250	-51.0	NULL	TRUE	0	1969-12-31 16:00:08.451	q8M86Fx0r	-51.0	1.495107764E9	-0.6702291758433747	7.47553857E8
-true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-335450417	1233327000	11	NULL	1	2	NULL	NULL	11	11	11	11.0	NULL	-3.35450417E8	1.233327E9	11.0	NULL	1.0	2.351	NULL	NULL	-3.35450432E8	NULL	1969-12-31 16:00:00.011	NULL	1969-12-27 18:49:09.583	1970-01-14 22:35:27	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-335450417	1233327000	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	dOYnqgaXoJ1P3ERwxe5N7	11.0	-6.70900834E8	-0.9999902065507035	-3.35450431E8
-true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-64615982	1803053750	11	NULL	1	2	NULL	8	11	11	11	11.0	NULL	-6.4615982E7	1.80305375E9	11.0	NULL	1.0	2.351	NULL	8.0	-6.4615984E7	NULL	1969-12-31 16:00:00.011	NULL	1969-12-30 22:03:04.018	1970-01-21 12:50:53.75	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-64615982	1803053750	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	8J5OB7K26PEV7kdbeHr3	11.0	-1.29231964E8	-0.9999902065507035	-6.4615983E7
-true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	890988972	-1862301000	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	8.90988972E8	-1.862301E9	8.0	NULL	1.0	15.892	NULL	NULL	8.9098899E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-10 23:29:48.972	1969-12-10 02:41:39	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	890988972	-1862301000	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	XylAH4	8.0	1.781977944E9	0.9893582466233818	8.90988993E8
-true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	930867246	1205399250	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	9.30867246E8	1.20539925E9	8.0	NULL	1.0	15.892	NULL	NULL	9.3086726E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-11 10:34:27.246	1970-01-14 14:49:59.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	930867246	1205399250	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	c1V8o1A	8.0	1.861734492E9	0.9893582466233818	9.30867265E8
-true	true	NULL	true	true	true	NULL	false	true	NULL	-14	-7196	NULL	-1552199500	-14	-7196	NULL	11	NULL	NULL	-14	-14	-14	-14.0	-7196.0	NULL	-1.5521995E9	-14.0	-7196.0	NULL	11.065	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.986	1969-12-31 15:59:52.804	NULL	1969-12-13 16:50:00.5	1969-12-31 15:59:46	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:11.065	NULL	NULL	-14	-7196	NULL	-1552199500	-14.0	-7196.0	NULL	0	1969-12-31 16:00:11.065	NULL	-14.0	NULL	-0.9906073556948704	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-21	-7196	NULL	1542429000	-21	-7196	NULL	-5	NULL	NULL	-21	-21	-21	-21.0	-7196.0	NULL	1.542429E9	-21.0	-7196.0	NULL	-4.1	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.979	1969-12-31 15:59:52.804	NULL	1970-01-18 12:27:09	1969-12-31 15:59:39	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:55.9	NULL	NULL	-21	-7196	NULL	1542429000	-21.0	-7196.0	NULL	0	1969-12-31 15:59:55.9	NULL	-21.0	NULL	-0.8366556385360561	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-24	-7196	NULL	829111000	-24	-7196	NULL	-7	NULL	NULL	-24	-24	-24	-24.0	-7196.0	NULL	8.29111E8	-24.0	-7196.0	NULL	-6.855	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.976	1969-12-31 15:59:52.804	NULL	1970-01-10 06:18:31	1969-12-31 15:59:36	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.145	NULL	NULL	-24	-7196	NULL	829111000	-24.0	-7196.0	NULL	0	1969-12-31 15:59:53.145	NULL	-24.0	NULL	0.9055783620066238	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-30	-200	NULL	1429852250	-30	-200	NULL	12	NULL	NULL	-30	-30	-30	-30.0	-200.0	NULL	1.42985225E9	-30.0	-200.0	NULL	12.935	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.97	1969-12-31 15:59:59.8	NULL	1970-01-17 05:10:52.25	1969-12-31 15:59:30	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 16:00:12.935	NULL	NULL	-30	-200	NULL	1429852250	-30.0	-200.0	NULL	0	1969-12-31 16:00:12.935	NULL	-30.0	NULL	0.9880316240928618	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	-2006216750	-36	-200	NULL	-15	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	-2.00621675E9	-36.0	-200.0	NULL	-14.252	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1969-12-08 10:43:03.25	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.748	NULL	NULL	-36	-200	NULL	-2006216750	-36.0	-200.0	NULL	0	1969-12-31 15:59:45.748	NULL	-36.0	NULL	0.9917788534431158	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	1599879000	-36	-200	NULL	-7	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	1.599879E9	-36.0	-200.0	NULL	-6.183	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1970-01-19 04:24:39	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.817	NULL	NULL	-36	-200	NULL	1599879000	-36.0	-200.0	NULL	0	1969-12-31 15:59:53.817	NULL	-36.0	NULL	0.9917788534431158	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-38	15601	NULL	-1858689000	-38	15601	NULL	-2	NULL	NULL	-38	-38	-38	-38.0	15601.0	NULL	-1.858689E9	-38.0	15601.0	NULL	-1.3860000000000001	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.962	1969-12-31 16:00:15.601	NULL	1969-12-10 03:41:51	1969-12-31 15:59:22	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:58.614	NULL	NULL	-38	15601	NULL	-1858689000	-38.0	15601.0	NULL	0	1969-12-31 15:59:58.614	NULL	-38.0	NULL	-0.2963685787093853	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-5	15601	NULL	612416000	-5	15601	NULL	4	NULL	NULL	-5	-5	-5	-5.0	15601.0	NULL	6.12416E8	-5.0	15601.0	NULL	4.679	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.995	1969-12-31 16:00:15.601	NULL	1970-01-07 18:06:56	1969-12-31 15:59:55	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 16:00:04.679	NULL	NULL	-5	15601	NULL	612416000	-5.0	15601.0	NULL	0	1969-12-31 16:00:04.679	NULL	-5.0	NULL	0.9589242746631385	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-50	-7196	NULL	-1031187250	-50	-7196	NULL	-6	NULL	NULL	-50	-50	-50	-50.0	-7196.0	NULL	-1.03118725E9	-50.0	-7196.0	NULL	-5.267	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.95	1969-12-31 15:59:52.804	NULL	1969-12-19 17:33:32.75	1969-12-31 15:59:10	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:54.733	NULL	NULL	-50	-7196	NULL	-1031187250	-50.0	-7196.0	NULL	0	1969-12-31 15:59:54.733	NULL	-50.0	NULL	0.26237485370392877	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-59	-7196	NULL	-1604890000	-59	-7196	NULL	13	NULL	NULL	-59	-59	-59	-59.0	-7196.0	NULL	-1.60489E9	-59.0	-7196.0	NULL	13.15	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.941	1969-12-31 15:59:52.804	NULL	1969-12-13 02:11:50	1969-12-31 15:59:01	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:13.15	NULL	NULL	-59	-7196	NULL	-1604890000	-59.0	-7196.0	NULL	0	1969-12-31 16:00:13.15	NULL	-59.0	NULL	-0.6367380071391379	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-60	-7196	NULL	1516314750	-60	-7196	NULL	-8	NULL	NULL	-60	-60	-60	-60.0	-7196.0	NULL	1.51631475E9	-60.0	-7196.0	NULL	-7.592	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.94	1969-12-31 15:59:52.804	NULL	1970-01-18 05:11:54.75	1969-12-31 15:59:00	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:52.408	NULL	NULL	-60	-7196	NULL	1516314750	-60.0	-7196.0	NULL	0	1969-12-31 15:59:52.408	NULL	-60.0	NULL	0.3048106211022167	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-8	-7196	NULL	-1849991500	-8	-7196	NULL	3	NULL	NULL	-8	-8	-8	-8.0	-7196.0	NULL	-1.8499915E9	-8.0	-7196.0	NULL	3.136	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.992	1969-12-31 15:59:52.804	NULL	1969-12-10 06:06:48.5	1969-12-31 15:59:52	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:03.136	NULL	NULL	-8	-7196	NULL	-1849991500	-8.0	-7196.0	NULL	0	1969-12-31 16:00:03.136	NULL	-8.0	NULL	-0.9893582466233818	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	20	15601	NULL	-362433250	20	15601	NULL	-15	NULL	NULL	20	20	20	20.0	15601.0	NULL	-3.6243325E8	20.0	15601.0	NULL	-14.871	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.02	1969-12-31 16:00:15.601	NULL	1969-12-27 11:19:26.75	1969-12-31 16:00:20	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.129	NULL	NULL	20	15601	NULL	-362433250	20.0	15601.0	NULL	0	1969-12-31 15:59:45.129	NULL	20.0	NULL	0.9129452507276277	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	48	15601	NULL	-795361000	48	15601	NULL	-10	NULL	NULL	48	48	48	48.0	15601.0	NULL	-7.95361E8	48.0	15601.0	NULL	-9.765	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.048	1969-12-31 16:00:15.601	NULL	1969-12-22 11:03:59	1969-12-31 16:00:48	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:50.235	NULL	NULL	48	15601	NULL	-795361000	48.0	15601.0	NULL	0	1969-12-31 15:59:50.235	NULL	48.0	NULL	-0.7682546613236668	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	5	-7196	NULL	-1015607500	5	-7196	NULL	10	NULL	NULL	5	5	5	5.0	-7196.0	NULL	-1.0156075E9	5.0	-7196.0	NULL	10.973	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.005	1969-12-31 15:59:52.804	NULL	1969-12-19 21:53:12.5	1969-12-31 16:00:05	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.973	NULL	NULL	5	-7196	NULL	-1015607500	5.0	-7196.0	NULL	0	1969-12-31 16:00:10.973	NULL	5.0	NULL	-0.9589242746631385	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	59	-7196	NULL	-1137754500	59	-7196	NULL	10	NULL	NULL	59	59	59	59.0	-7196.0	NULL	-1.1377545E9	59.0	-7196.0	NULL	10.956	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.059	1969-12-31 15:59:52.804	NULL	1969-12-18 11:57:25.5	1969-12-31 16:00:59	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.956	NULL	NULL	59	-7196	NULL	-1137754500	59.0	-7196.0	NULL	0	1969-12-31 16:00:10.956	NULL	59.0	NULL	0.6367380071391379	NULL
+true	NULL	true	true	true	NULL	false	false	true	true	-51	NULL	773600971	1053923250	-51	NULL	0	8	NULL	2	-51	-51	-51	-51.0	NULL	7.73600971E8	1.05392325E9	-51.0	NULL	0.0	8.451	NULL	2.0	7.7360096E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 14:53:20.971	1970-01-12 20:45:23.25	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	773600971	1053923250	-51.0	NULL	FALSE	0	1969-12-31 16:00:08.451	2yK4Bx76O	2yK4Bx76O	2yK4Bx76O	-51.0	1.547201942E9	-0.6702291758433747	7.73600971E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-102936434	-1312782750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-1.02936434E8	-1.31278275E9	8.0	NULL	0.0	15.892	NULL	NULL	-1.02936432E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-30 11:24:23.566	1969-12-16 11:20:17.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-102936434	-1312782750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	eJROSNhugc3kQR7Pb	eJROSNhugc	eJROSNhugc	8.0	-2.05872868E8	0.9893582466233818	-1.02936434E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-661621138	-931392750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-6.61621138E8	-9.3139275E8	8.0	NULL	0.0	15.892	NULL	NULL	-6.6162112E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-24 00:12:58.862	1969-12-20 21:16:47.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-661621138	-931392750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	L15l8i5k558tBcDV20	L15l8i5k55	L15l8i5k55	8.0	-1.323242276E9	0.9893582466233818	-6.61621138E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-669632311	1588591250	8	NULL	0	15	NULL	3	8	8	8	8.0	NULL	-6.69632311E8	1.58859125E9	8.0	NULL	0.0	15.892	NULL	3.0	-6.6963232E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-23 21:59:27.689	1970-01-19 01:16:31.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-669632311	1588591250	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	3r3sDvfUkG0yTP3LnX5mNQRr	3r3sDvfUkG	3r3sDvfUkG	8.0	-1.339264622E9	0.9893582466233818	-6.69632311E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	805179664	868161500	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	8.05179664E8	8.681615E8	8.0	NULL	0.0	15.892	NULL	NULL	8.0517965E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-09 23:39:39.664	1970-01-10 17:09:21.5	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	805179664	868161500	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	e005B5q	e005B5q	e005B5q	8.0	1.610359328E9	0.9893582466233818	8.05179664E8
+true	NULL	true	true	true	NULL	true	false	true	true	-51	NULL	747553882	-1930467250	-51	NULL	1	8	NULL	NULL	-51	-51	-51	-51.0	NULL	7.47553882E8	-1.93046725E9	-51.0	NULL	1.0	8.451	NULL	NULL	7.4755386E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 07:39:13.882	1969-12-09 07:45:32.75	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	747553882	-1930467250	-51.0	NULL	TRUE	0	1969-12-31 16:00:08.451	q8M86Fx0r	q8M86Fx0r	q8M86Fx0r	-51.0	1.495107764E9	-0.6702291758433747	7.47553883E8
+true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-335450417	1233327000	11	NULL	1	2	NULL	NULL	11	11	11	11.0	NULL	-3.35450417E8	1.233327E9	11.0	NULL	1.0	2.351	NULL	NULL	-3.35450432E8	NULL	1969-12-31 16:00:00.011	NULL	1969-12-27 18:49:09.583	1970-01-14 22:35:27	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-335450417	1233327000	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	dOYnqgaXoJ1P3ERwxe5N7	dOYnqgaXoJ	dOYnqgaXoJ	11.0	-6.70900834E8	-0.9999902065507035	-3.35450416E8
+true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-64615982	1803053750	11	NULL	1	2	NULL	8	11	11	11	11.0	NULL	-6.4615982E7	1.80305375E9	11.0	NULL	1.0	2.351	NULL	8.0	-6.4615984E7	NULL	1969-12-31 16:00:00.011	NULL	1969-12-30 22:03:04.018	1970-01-21 12:50:53.75	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-64615982	1803053750	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	8J5OB7K26PEV7kdbeHr3	8J5OB7K26P	8J5OB7K26P	11.0	-1.29231964E8	-0.9999902065507035	-6.4615981E7
+true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	890988972	-1862301000	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	8.90988972E8	-1.862301E9	8.0	NULL	1.0	15.892	NULL	NULL	8.9098899E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-10 23:29:48.972	1969-12-10 02:41:39	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	890988972	-1862301000	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	XylAH4	XylAH4	XylAH4	8.0	1.781977944E9	0.9893582466233818	8.90988973E8
+true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	930867246	1205399250	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	9.30867246E8	1.20539925E9	8.0	NULL	1.0	15.892	NULL	NULL	9.3086726E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-11 10:34:27.246	1970-01-14 14:49:59.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	930867246	1205399250	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	c1V8o1A	c1V8o1A	c1V8o1A	8.0	1.861734492E9	0.9893582466233818	9.30867247E8
+true	true	NULL	true	true	true	NULL	false	true	NULL	-14	-7196	NULL	-1552199500	-14	-7196	NULL	11	NULL	NULL	-14	-14	-14	-14.0	-7196.0	NULL	-1.5521995E9	-14.0	-7196.0	NULL	11.065	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.986	1969-12-31 15:59:52.804	NULL	1969-12-13 16:50:00.5	1969-12-31 15:59:46	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:11.065	NULL	NULL	-14	-7196	NULL	-1552199500	-14.0	-7196.0	NULL	0	1969-12-31 16:00:11.065	NULL	NULL	NULL	-14.0	NULL	-0.9906073556948704	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-21	-7196	NULL	1542429000	-21	-7196	NULL	-4	NULL	NULL	-21	-21	-21	-21.0	-7196.0	NULL	1.542429E9	-21.0	-7196.0	NULL	-4.1	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.979	1969-12-31 15:59:52.804	NULL	1970-01-18 12:27:09	1969-12-31 15:59:39	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:55.9	NULL	NULL	-21	-7196	NULL	1542429000	-21.0	-7196.0	NULL	0	1969-12-31 15:59:55.9	NULL	NULL	NULL	-21.0	NULL	-0.8366556385360561	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-24	-7196	NULL	829111000	-24	-7196	NULL	-6	NULL	NULL	-24	-24	-24	-24.0	-7196.0	NULL	8.29111E8	-24.0	-7196.0	NULL	-6.855	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.976	1969-12-31 15:59:52.804	NULL	1970-01-10 06:18:31	1969-12-31 15:59:36	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.145	NULL	NULL	-24	-7196	NULL	829111000	-24.0	-7196.0	NULL	0	1969-12-31 15:59:53.145	NULL	NULL	NULL	-24.0	NULL	0.9055783620066238	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-30	-200	NULL	1429852250	-30	-200	NULL	12	NULL	NULL	-30	-30	-30	-30.0	-200.0	NULL	1.42985225E9	-30.0	-200.0	NULL	12.935	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.97	1969-12-31 15:59:59.8	NULL	1970-01-17 05:10:52.25	1969-12-31 15:59:30	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 16:00:12.935	NULL	NULL	-30	-200	NULL	1429852250	-30.0	-200.0	NULL	0	1969-12-31 16:00:12.935	NULL	NULL	NULL	-30.0	NULL	0.9880316240928618	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	-2006216750	-36	-200	NULL	-14	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	-2.00621675E9	-36.0	-200.0	NULL	-14.252	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1969-12-08 10:43:03.25	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.748	NULL	NULL	-36	-200	NULL	-2006216750	-36.0	-200.0	NULL	0	1969-12-31 15:59:45.748	NULL	NULL	NULL	-36.0	NULL	0.9917788534431158	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	1599879000	-36	-200	NULL	-6	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	1.599879E9	-36.0	-200.0	NULL	-6.183	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1970-01-19 04:24:39	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.817	NULL	NULL	-36	-200	NULL	1599879000	-36.0	-200.0	NULL	0	1969-12-31 15:59:53.817	NULL	NULL	NULL	-36.0	NULL	0.9917788534431158	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-38	15601	NULL	-1858689000	-38	15601	NULL	-1	NULL	NULL	-38	-38	-38	-38.0	15601.0	NULL	-1.858689E9	-38.0	15601.0	NULL	-1.386	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.962	1969-12-31 16:00:15.601	NULL	1969-12-10 03:41:51	1969-12-31 15:59:22	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:58.614	NULL	NULL	-38	15601	NULL	-1858689000	-38.0	15601.0	NULL	0	1969-12-31 15:59:58.614	NULL	NULL	NULL	-38.0	NULL	-0.2963685787093853	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-5	15601	NULL	612416000	-5	15601	NULL	4	NULL	NULL	-5	-5	-5	-5.0	15601.0	NULL	6.12416E8	-5.0	15601.0	NULL	4.679	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.995	1969-12-31 16:00:15.601	NULL	1970-01-07 18:06:56	1969-12-31 15:59:55	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 16:00:04.679	NULL	NULL	-5	15601	NULL	612416000	-5.0	15601.0	NULL	0	1969-12-31 16:00:04.679	NULL	NULL	NULL	-5.0	NULL	0.9589242746631385	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-50	-7196	NULL	-1031187250	-50	-7196	NULL	-5	NULL	NULL	-50	-50	-50	-50.0	-7196.0	NULL	-1.03118725E9	-50.0	-7196.0	NULL	-5.267	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.95	1969-12-31 15:59:52.804	NULL	1969-12-19 17:33:32.75	1969-12-31 15:59:10	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:54.733	NULL	NULL	-50	-7196	NULL	-1031187250	-50.0	-7196.0	NULL	0	1969-12-31 15:59:54.733	NULL	NULL	NULL	-50.0	NULL	0.26237485370392877	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-59	-7196	NULL	-1604890000	-59	-7196	NULL	13	NULL	NULL	-59	-59	-59	-59.0	-7196.0	NULL	-1.60489E9	-59.0	-7196.0	NULL	13.15	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.941	1969-12-31 15:59:52.804	NULL	1969-12-13 02:11:50	1969-12-31 15:59:01	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:13.15	NULL	NULL	-59	-7196	NULL	-1604890000	-59.0	-7196.0	NULL	0	1969-12-31 16:00:13.15	NULL	NULL	NULL	-59.0	NULL	-0.6367380071391379	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-60	-7196	NULL	1516314750	-60	-7196	NULL	-7	NULL	NULL	-60	-60	-60	-60.0	-7196.0	NULL	1.51631475E9	-60.0	-7196.0	NULL	-7.592	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.94	1969-12-31 15:59:52.804	NULL	1970-01-18 05:11:54.75	1969-12-31 15:59:00	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:52.408	NULL	NULL	-60	-7196	NULL	1516314750	-60.0	-7196.0	NULL	0	1969-12-31 15:59:52.408	NULL	NULL	NULL	-60.0	NULL	0.3048106211022167	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-8	-7196	NULL	-1849991500	-8	-7196	NULL	3	NULL	NULL	-8	-8	-8	-8.0	-7196.0	NULL	-1.8499915E9	-8.0	-7196.0	NULL	3.136	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.992	1969-12-31 15:59:52.804	NULL	1969-12-10 06:06:48.5	1969-12-31 15:59:52	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:03.136	NULL	NULL	-8	-7196	NULL	-1849991500	-8.0	-7196.0	NULL	0	1969-12-31 16:00:03.136	NULL	NULL	NULL	-8.0	NULL	-0.9893582466233818	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	20	15601	NULL	-362433250	20	15601	NULL	-14	NULL	NULL	20	20	20	20.0	15601.0	NULL	-3.6243325E8	20.0	15601.0	NULL	-14.871	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.02	1969-12-31 16:00:15.601	NULL	1969-12-27 11:19:26.75	1969-12-31 16:00:20	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.129	NULL	NULL	20	15601	NULL	-362433250	20.0	15601.0	NULL	0	1969-12-31 15:59:45.129	NULL	NULL	NULL	20.0	NULL	0.9129452507276277	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	48	15601	NULL	-795361000	48	15601	NULL	-9	NULL	NULL	48	48	48	48.0	15601.0	NULL	-7.95361E8	48.0	15601.0	NULL	-9.765	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.048	1969-12-31 16:00:15.601	NULL	1969-12-22 11:03:59	1969-12-31 16:00:48	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:50.235	NULL	NULL	48	15601	NULL	-795361000	48.0	15601.0	NULL	0	1969-12-31 15:59:50.235	NULL	NULL	NULL	48.0	NULL	-0.7682546613236668	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	5	-7196	NULL	-1015607500	5	-7196	NULL	10	NULL	NULL	5	5	5	5.0	-7196.0	NULL	-1.0156075E9	5.0	-7196.0	NULL	10.973	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.005	1969-12-31 15:59:52.804	NULL	1969-12-19 21:53:12.5	1969-12-31 16:00:05	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.973	NULL	NULL	5	-7196	NULL	-1015607500	5.0	-7196.0	NULL	0	1969-12-31 16:00:10.973	NULL	NULL	NULL	5.0	NULL	-0.9589242746631385	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	59	-7196	NULL	-1137754500	59	-7196	NULL	10	NULL	NULL	59	59	59	59.0	-7196.0	NULL	-1.1377545E9	59.0	-7196.0	NULL	10.956	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.059	1969-12-31 15:59:52.804	NULL	1969-12-18 11:57:25.5	1969-12-31 16:00:59	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.956	NULL	NULL	59	-7196	NULL	-1137754500	59.0	-7196.0	NULL	0	1969-12-31 16:00:10.956	NULL	NULL	NULL	59.0	NULL	0.6367380071391379	NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out b/ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out
index 744bfb3..af5f0de 100644
--- a/ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out
+++ b/ql/src/test/results/clientpositive/vector_char_mapjoin1.q.out
@@ -371,6 +371,7 @@ STAGE PLANS:
                     value expressions: _col1 (type: char(10)), _col2 (type: int), _col3 (type: string)
       Local Work:
         Map Reduce Local Work
+      Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: char(10)), VALUE._col1 (type: int), VALUE._col2 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/test/results/clientpositive/vector_varchar_mapjoin1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_varchar_mapjoin1.q.out b/ql/src/test/results/clientpositive/vector_varchar_mapjoin1.q.out
index 2994bff..936ef78 100644
--- a/ql/src/test/results/clientpositive/vector_varchar_mapjoin1.q.out
+++ b/ql/src/test/results/clientpositive/vector_varchar_mapjoin1.q.out
@@ -359,6 +359,7 @@ STAGE PLANS:
                     value expressions: _col1 (type: varchar(10)), _col2 (type: int), _col3 (type: string)
       Local Work:
         Map Reduce Local Work
+      Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: varchar(10)), VALUE._col1 (type: int), VALUE._col2 (type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/2de6fda0/ql/src/test/results/clientpositive/vectorized_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_casts.q.out b/ql/src/test/results/clientpositive/vectorized_casts.q.out
index 1113453..3c28064 100644
--- a/ql/src/test/results/clientpositive/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_casts.q.out
@@ -1,5 +1,6 @@
 PREHOOK: query: -- SORT_QUERY_RESULTS
 
+-- Currently, vectorization is not supported in fetch task (hive.fetch.task.conversion=none)
 -- Test type casting in vectorized mode to verify end-to-end functionality.
 
 explain 
@@ -65,6 +66,8 @@ select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -76,6 +79,7 @@ where cbigint % 250 = 0
 PREHOOK: type: QUERY
 POSTHOOK: query: -- SORT_QUERY_RESULTS
 
+-- Currently, vectorization is not supported in fetch task (hive.fetch.task.conversion=none)
 -- Test type casting in vectorized mode to verify end-to-end functionality.
 
 explain 
@@ -141,6 +145,8 @@ select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -165,8 +171,8 @@ STAGE PLANS:
               predicate: ((cbigint % 250) = 0) (type: boolean)
               Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), UDFToBoolean((cbigint * 0)) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (type: int), UDFToInteger(cbigint) (type: int), UDFToInteger(cfloat) (type: int), UDFToInteger(cdouble) (type: int), UDFToInteger(cboolean1) (type: int), UDFToInteger(ctimestamp1) (type: int), UDFToInteger(cstring1) (type: int), UDFToInteger(substr(cstring1, 1, 1)) (type: int), UDFToByte(cfloat) (type: tinyint), UDFToShort(cfloat) (type: smallint), UDFToLong(cfloat) (type: bigint), UDFToDouble(ctinyint) (type: double), UDFToDouble(csmallint) (type: double), UDFToDouble(cint) (type: double), UDFToD
 ouble(cbigint) (type: double), UDFToDouble(cfloat) (type: double), cdouble (type: double), UDFToDouble(cboolean1) (type: double), UDFToDouble(ctimestamp1) (type: double), UDFToDouble(cstring1) (type: double), UDFToDouble(substr(cstring1, 1, 1)) (type: double), UDFToFloat(cint) (type: float), UDFToFloat(cdouble) (type: float), CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp), CAST( substr(cstring1, 1, 1) AS TIMESTAMP) (type: timestamp), UDFToString(ctinyint) (type: string), UDFToString(csmallint) (type: string), UDFToString(cint) (type: string), UDFToString(cbigint) (type: string
 ), UDFToString(cfloat) (type: string), UDFToString(cdouble) (type: string), UDFToString(cboolean1) (type: string), UDFToString((cbigint * 0)) (type: string), UDFToString(ctimestamp1) (type: string), cstring1 (type: string), UDFToFloat(UDFToInteger(cfloat)) (type: float), UDFToDouble((cint * 2)) (type: double), UDFToString(sin(cfloat)) (type: string), (UDFToDouble(UDFToFloat(cint)) + UDFToDouble(cboolean1)) (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col57, _col58, _col59
+                expressions: UDFToBoolean(ctinyint) (type: boolean), UDFToBoolean(csmallint) (type: boolean), UDFToBoolean(cint) (type: boolean), UDFToBoolean(cbigint) (type: boolean), UDFToBoolean(cfloat) (type: boolean), UDFToBoolean(cdouble) (type: boolean), cboolean1 (type: boolean), UDFToBoolean((cbigint * 0)) (type: boolean), UDFToBoolean(ctimestamp1) (type: boolean), UDFToBoolean(cstring1) (type: boolean), UDFToInteger(ctinyint) (type: int), UDFToInteger(csmallint) (type: int), cint (type: int), UDFToInteger(cbigint) (type: int), UDFToInteger(cfloat) (type: int), UDFToInteger(cdouble) (type: int), UDFToInteger(cboolean1) (type: int), UDFToInteger(ctimestamp1) (type: int), UDFToInteger(cstring1) (type: int), UDFToInteger(substr(cstring1, 1, 1)) (type: int), UDFToByte(cfloat) (type: tinyint), UDFToShort(cfloat) (type: smallint), UDFToLong(cfloat) (type: bigint), UDFToDouble(ctinyint) (type: double), UDFToDouble(csmallint) (type: double), UDFToDouble(cint) (type: double), UDFToD
 ouble(cbigint) (type: double), UDFToDouble(cfloat) (type: double), cdouble (type: double), UDFToDouble(cboolean1) (type: double), UDFToDouble(ctimestamp1) (type: double), UDFToDouble(cstring1) (type: double), UDFToDouble(substr(cstring1, 1, 1)) (type: double), UDFToFloat(cint) (type: float), UDFToFloat(cdouble) (type: float), CAST( ctinyint AS TIMESTAMP) (type: timestamp), CAST( csmallint AS TIMESTAMP) (type: timestamp), CAST( cint AS TIMESTAMP) (type: timestamp), CAST( cbigint AS TIMESTAMP) (type: timestamp), CAST( cfloat AS TIMESTAMP) (type: timestamp), CAST( cdouble AS TIMESTAMP) (type: timestamp), CAST( cboolean1 AS TIMESTAMP) (type: timestamp), CAST( (cbigint * 0) AS TIMESTAMP) (type: timestamp), ctimestamp1 (type: timestamp), CAST( cstring1 AS TIMESTAMP) (type: timestamp), CAST( substr(cstring1, 1, 1) AS TIMESTAMP) (type: timestamp), UDFToString(ctinyint) (type: string), UDFToString(csmallint) (type: string), UDFToString(cint) (type: string), UDFToString(cbigint) (type: string
 ), UDFToString(cfloat) (type: string), UDFToString(cdouble) (type: string), UDFToString(cboolean1) (type: string), UDFToString((cbigint * 0)) (type: string), UDFToString(ctimestamp1) (type: string), cstring1 (type: string), UDFToString(CAST( cstring1 AS CHAR(10)) (type: string), UDFToString(CAST( cstring1 AS varchar(10))) (type: string), UDFToFloat(UDFToInteger(cfloat)) (type: float), UDFToDouble((cint * 2)) (type: double), UDFToString(sin(cfloat)) (type: string), (UDFToDouble(UDFToFloat(cint)) + UDFToDouble(cboolean1)) (type: double)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51, _col52, _col53, _col54, _col55, _col56, _col57, _col58, _col59, _col60, _col61
                 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
@@ -245,6 +251,8 @@ PREHOOK: query: select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -318,6 +326,8 @@ POSTHOOK: query: select
   ,cast (cbigint * 0 as string)
   ,cast (ctimestamp1 as string)
   ,cast (cstring1 as string)
+  ,cast (cast (cstring1 as char(10)) as string)
+  ,cast (cast (cstring1 as varchar(10)) as string)
 -- nested and expression arguments
   ,cast (cast (cfloat as int) as float)
   ,cast (cint * 2 as double)
@@ -329,29 +339,29 @@ where cbigint % 250 = 0
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-true	NULL	true	true	true	NULL	false	false	true	true	-51	NULL	773600971	1053923250	-51	NULL	0	8	NULL	2	-51	-51	-51	-51.0	NULL	7.73600971E8	1.05392325E9	-51.0	NULL	0.0	8.451	NULL	2.0	7.7360096E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 14:53:20.971	1970-01-12 20:45:23.25	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	773600971	1053923250	-51.0	NULL	FALSE	0	1969-12-31 16:00:08.451	2yK4Bx76O	-51.0	1.547201942E9	-0.6702291758433747	7.73600971E8
-true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-102936434	-1312782750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-1.02936434E8	-1.31278275E9	8.0	NULL	0.0	15.892	NULL	NULL	-1.02936432E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-30 11:24:23.566	1969-12-16 11:20:17.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-102936434	-1312782750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	eJROSNhugc3kQR7Pb	8.0	-2.05872868E8	0.9893582466233818	-1.02936434E8
-true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-661621138	-931392750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-6.61621138E8	-9.3139275E8	8.0	NULL	0.0	15.892	NULL	NULL	-6.6162112E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-24 00:12:58.862	1969-12-20 21:16:47.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-661621138	-931392750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	L15l8i5k558tBcDV20	8.0	-1.323242276E9	0.9893582466233818	-6.61621138E8
-true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-669632311	1588591250	8	NULL	0	15	NULL	3	8	8	8	8.0	NULL	-6.69632311E8	1.58859125E9	8.0	NULL	0.0	15.892	NULL	3.0	-6.6963232E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-23 21:59:27.689	1970-01-19 01:16:31.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-669632311	1588591250	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	3r3sDvfUkG0yTP3LnX5mNQRr	8.0	-1.339264622E9	0.9893582466233818	-6.69632311E8
-true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	805179664	868161500	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	8.05179664E8	8.681615E8	8.0	NULL	0.0	15.892	NULL	NULL	8.0517965E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-09 23:39:39.664	1970-01-10 17:09:21.5	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	805179664	868161500	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	e005B5q	8.0	1.610359328E9	0.9893582466233818	8.05179664E8
-true	NULL	true	true	true	NULL	true	false	true	true	-51	NULL	747553882	-1930467250	-51	NULL	1	8	NULL	NULL	-51	-51	-51	-51.0	NULL	7.47553882E8	-1.93046725E9	-51.0	NULL	1.0	8.451	NULL	NULL	7.4755386E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 07:39:13.882	1969-12-09 07:45:32.75	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	747553882	-1930467250	-51.0	NULL	TRUE	0	1969-12-31 16:00:08.451	q8M86Fx0r	-51.0	1.495107764E9	-0.6702291758433747	7.47553883E8
-true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-335450417	1233327000	11	NULL	1	2	NULL	NULL	11	11	11	11.0	NULL	-3.35450417E8	1.233327E9	11.0	NULL	1.0	2.351	NULL	NULL	-3.35450432E8	NULL	1969-12-31 16:00:00.011	NULL	1969-12-27 18:49:09.583	1970-01-14 22:35:27	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-335450417	1233327000	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	dOYnqgaXoJ1P3ERwxe5N7	11.0	-6.70900834E8	-0.9999902065507035	-3.35450416E8
-true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-64615982	1803053750	11	NULL	1	2	NULL	8	11	11	11	11.0	NULL	-6.4615982E7	1.80305375E9	11.0	NULL	1.0	2.351	NULL	8.0	-6.4615984E7	NULL	1969-12-31 16:00:00.011	NULL	1969-12-30 22:03:04.018	1970-01-21 12:50:53.75	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-64615982	1803053750	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	8J5OB7K26PEV7kdbeHr3	11.0	-1.29231964E8	-0.9999902065507035	-6.4615981E7
-true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	890988972	-1862301000	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	8.90988972E8	-1.862301E9	8.0	NULL	1.0	15.892	NULL	NULL	8.9098899E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-10 23:29:48.972	1969-12-10 02:41:39	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	890988972	-1862301000	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	XylAH4	8.0	1.781977944E9	0.9893582466233818	8.90988973E8
-true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	930867246	1205399250	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	9.30867246E8	1.20539925E9	8.0	NULL	1.0	15.892	NULL	NULL	9.3086726E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-11 10:34:27.246	1970-01-14 14:49:59.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	930867246	1205399250	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	c1V8o1A	8.0	1.861734492E9	0.9893582466233818	9.30867247E8
-true	true	NULL	true	true	true	NULL	false	true	NULL	-14	-7196	NULL	-1552199500	-14	-7196	NULL	11	NULL	NULL	-14	-14	-14	-14.0	-7196.0	NULL	-1.5521995E9	-14.0	-7196.0	NULL	11.065	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.986	1969-12-31 15:59:52.804	NULL	1969-12-13 16:50:00.5	1969-12-31 15:59:46	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:11.065	NULL	NULL	-14	-7196	NULL	-1552199500	-14.0	-7196.0	NULL	0	1969-12-31 16:00:11.065	NULL	-14.0	NULL	-0.9906073556948704	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-21	-7196	NULL	1542429000	-21	-7196	NULL	-4	NULL	NULL	-21	-21	-21	-21.0	-7196.0	NULL	1.542429E9	-21.0	-7196.0	NULL	-4.1	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.979	1969-12-31 15:59:52.804	NULL	1970-01-18 12:27:09	1969-12-31 15:59:39	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:55.9	NULL	NULL	-21	-7196	NULL	1542429000	-21.0	-7196.0	NULL	0	1969-12-31 15:59:55.9	NULL	-21.0	NULL	-0.8366556385360561	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-24	-7196	NULL	829111000	-24	-7196	NULL	-6	NULL	NULL	-24	-24	-24	-24.0	-7196.0	NULL	8.29111E8	-24.0	-7196.0	NULL	-6.855	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.976	1969-12-31 15:59:52.804	NULL	1970-01-10 06:18:31	1969-12-31 15:59:36	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.145	NULL	NULL	-24	-7196	NULL	829111000	-24.0	-7196.0	NULL	0	1969-12-31 15:59:53.145	NULL	-24.0	NULL	0.9055783620066238	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-30	-200	NULL	1429852250	-30	-200	NULL	12	NULL	NULL	-30	-30	-30	-30.0	-200.0	NULL	1.42985225E9	-30.0	-200.0	NULL	12.935	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.97	1969-12-31 15:59:59.8	NULL	1970-01-17 05:10:52.25	1969-12-31 15:59:30	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 16:00:12.935	NULL	NULL	-30	-200	NULL	1429852250	-30.0	-200.0	NULL	0	1969-12-31 16:00:12.935	NULL	-30.0	NULL	0.9880316240928618	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	-2006216750	-36	-200	NULL	-14	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	-2.00621675E9	-36.0	-200.0	NULL	-14.252	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1969-12-08 10:43:03.25	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.748	NULL	NULL	-36	-200	NULL	-2006216750	-36.0	-200.0	NULL	0	1969-12-31 15:59:45.748	NULL	-36.0	NULL	0.9917788534431158	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	1599879000	-36	-200	NULL	-6	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	1.599879E9	-36.0	-200.0	NULL	-6.183	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1970-01-19 04:24:39	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.817	NULL	NULL	-36	-200	NULL	1599879000	-36.0	-200.0	NULL	0	1969-12-31 15:59:53.817	NULL	-36.0	NULL	0.9917788534431158	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-38	15601	NULL	-1858689000	-38	15601	NULL	-1	NULL	NULL	-38	-38	-38	-38.0	15601.0	NULL	-1.858689E9	-38.0	15601.0	NULL	-1.386	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.962	1969-12-31 16:00:15.601	NULL	1969-12-10 03:41:51	1969-12-31 15:59:22	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:58.614	NULL	NULL	-38	15601	NULL	-1858689000	-38.0	15601.0	NULL	0	1969-12-31 15:59:58.614	NULL	-38.0	NULL	-0.2963685787093853	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-5	15601	NULL	612416000	-5	15601	NULL	4	NULL	NULL	-5	-5	-5	-5.0	15601.0	NULL	6.12416E8	-5.0	15601.0	NULL	4.679	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.995	1969-12-31 16:00:15.601	NULL	1970-01-07 18:06:56	1969-12-31 15:59:55	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 16:00:04.679	NULL	NULL	-5	15601	NULL	612416000	-5.0	15601.0	NULL	0	1969-12-31 16:00:04.679	NULL	-5.0	NULL	0.9589242746631385	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-50	-7196	NULL	-1031187250	-50	-7196	NULL	-5	NULL	NULL	-50	-50	-50	-50.0	-7196.0	NULL	-1.03118725E9	-50.0	-7196.0	NULL	-5.267	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.95	1969-12-31 15:59:52.804	NULL	1969-12-19 17:33:32.75	1969-12-31 15:59:10	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:54.733	NULL	NULL	-50	-7196	NULL	-1031187250	-50.0	-7196.0	NULL	0	1969-12-31 15:59:54.733	NULL	-50.0	NULL	0.26237485370392877	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-59	-7196	NULL	-1604890000	-59	-7196	NULL	13	NULL	NULL	-59	-59	-59	-59.0	-7196.0	NULL	-1.60489E9	-59.0	-7196.0	NULL	13.15	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.941	1969-12-31 15:59:52.804	NULL	1969-12-13 02:11:50	1969-12-31 15:59:01	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:13.15	NULL	NULL	-59	-7196	NULL	-1604890000	-59.0	-7196.0	NULL	0	1969-12-31 16:00:13.15	NULL	-59.0	NULL	-0.6367380071391379	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-60	-7196	NULL	1516314750	-60	-7196	NULL	-7	NULL	NULL	-60	-60	-60	-60.0	-7196.0	NULL	1.51631475E9	-60.0	-7196.0	NULL	-7.592	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.94	1969-12-31 15:59:52.804	NULL	1970-01-18 05:11:54.75	1969-12-31 15:59:00	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:52.408	NULL	NULL	-60	-7196	NULL	1516314750	-60.0	-7196.0	NULL	0	1969-12-31 15:59:52.408	NULL	-60.0	NULL	0.3048106211022167	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	-8	-7196	NULL	-1849991500	-8	-7196	NULL	3	NULL	NULL	-8	-8	-8	-8.0	-7196.0	NULL	-1.8499915E9	-8.0	-7196.0	NULL	3.136	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.992	1969-12-31 15:59:52.804	NULL	1969-12-10 06:06:48.5	1969-12-31 15:59:52	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:03.136	NULL	NULL	-8	-7196	NULL	-1849991500	-8.0	-7196.0	NULL	0	1969-12-31 16:00:03.136	NULL	-8.0	NULL	-0.9893582466233818	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	20	15601	NULL	-362433250	20	15601	NULL	-14	NULL	NULL	20	20	20	20.0	15601.0	NULL	-3.6243325E8	20.0	15601.0	NULL	-14.871	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.02	1969-12-31 16:00:15.601	NULL	1969-12-27 11:19:26.75	1969-12-31 16:00:20	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.129	NULL	NULL	20	15601	NULL	-362433250	20.0	15601.0	NULL	0	1969-12-31 15:59:45.129	NULL	20.0	NULL	0.9129452507276277	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	48	15601	NULL	-795361000	48	15601	NULL	-9	NULL	NULL	48	48	48	48.0	15601.0	NULL	-7.95361E8	48.0	15601.0	NULL	-9.765	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.048	1969-12-31 16:00:15.601	NULL	1969-12-22 11:03:59	1969-12-31 16:00:48	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:50.235	NULL	NULL	48	15601	NULL	-795361000	48.0	15601.0	NULL	0	1969-12-31 15:59:50.235	NULL	48.0	NULL	-0.7682546613236668	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	5	-7196	NULL	-1015607500	5	-7196	NULL	10	NULL	NULL	5	5	5	5.0	-7196.0	NULL	-1.0156075E9	5.0	-7196.0	NULL	10.973	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.005	1969-12-31 15:59:52.804	NULL	1969-12-19 21:53:12.5	1969-12-31 16:00:05	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.973	NULL	NULL	5	-7196	NULL	-1015607500	5.0	-7196.0	NULL	0	1969-12-31 16:00:10.973	NULL	5.0	NULL	-0.9589242746631385	NULL
-true	true	NULL	true	true	true	NULL	false	true	NULL	59	-7196	NULL	-1137754500	59	-7196	NULL	10	NULL	NULL	59	59	59	59.0	-7196.0	NULL	-1.1377545E9	59.0	-7196.0	NULL	10.956	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.059	1969-12-31 15:59:52.804	NULL	1969-12-18 11:57:25.5	1969-12-31 16:00:59	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.956	NULL	NULL	59	-7196	NULL	-1137754500	59.0	-7196.0	NULL	0	1969-12-31 16:00:10.956	NULL	59.0	NULL	0.6367380071391379	NULL
+true	NULL	true	true	true	NULL	false	false	true	true	-51	NULL	773600971	1053923250	-51	NULL	0	8	NULL	2	-51	-51	-51	-51.0	NULL	7.73600971E8	1.05392325E9	-51.0	NULL	0.0	8.451	NULL	2.0	7.7360096E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 14:53:20.971	1970-01-12 20:45:23.25	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	773600971	1053923250	-51.0	NULL	FALSE	0	1969-12-31 16:00:08.451	2yK4Bx76O	2yK4Bx76O	2yK4Bx76O	-51.0	1.547201942E9	-0.6702291758433747	7.73600971E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-102936434	-1312782750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-1.02936434E8	-1.31278275E9	8.0	NULL	0.0	15.892	NULL	NULL	-1.02936432E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-30 11:24:23.566	1969-12-16 11:20:17.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-102936434	-1312782750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	eJROSNhugc3kQR7Pb	eJROSNhugc	eJROSNhugc	8.0	-2.05872868E8	0.9893582466233818	-1.02936434E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-661621138	-931392750	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	-6.61621138E8	-9.3139275E8	8.0	NULL	0.0	15.892	NULL	NULL	-6.6162112E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-24 00:12:58.862	1969-12-20 21:16:47.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-661621138	-931392750	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	L15l8i5k558tBcDV20	L15l8i5k55	L15l8i5k55	8.0	-1.323242276E9	0.9893582466233818	-6.61621138E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	-669632311	1588591250	8	NULL	0	15	NULL	3	8	8	8	8.0	NULL	-6.69632311E8	1.58859125E9	8.0	NULL	0.0	15.892	NULL	3.0	-6.6963232E8	NULL	1969-12-31 16:00:00.008	NULL	1969-12-23 21:59:27.689	1970-01-19 01:16:31.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	-669632311	1588591250	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	3r3sDvfUkG0yTP3LnX5mNQRr	3r3sDvfUkG	3r3sDvfUkG	8.0	-1.339264622E9	0.9893582466233818	-6.69632311E8
+true	NULL	true	true	true	NULL	false	false	true	true	8	NULL	805179664	868161500	8	NULL	0	15	NULL	NULL	8	8	8	8.0	NULL	8.05179664E8	8.681615E8	8.0	NULL	0.0	15.892	NULL	NULL	8.0517965E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-09 23:39:39.664	1970-01-10 17:09:21.5	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	805179664	868161500	8.0	NULL	FALSE	0	1969-12-31 16:00:15.892	e005B5q	e005B5q	e005B5q	8.0	1.610359328E9	0.9893582466233818	8.05179664E8
+true	NULL	true	true	true	NULL	true	false	true	true	-51	NULL	747553882	-1930467250	-51	NULL	1	8	NULL	NULL	-51	-51	-51	-51.0	NULL	7.47553882E8	-1.93046725E9	-51.0	NULL	1.0	8.451	NULL	NULL	7.4755386E8	NULL	1969-12-31 15:59:59.949	NULL	1970-01-09 07:39:13.882	1969-12-09 07:45:32.75	1969-12-31 15:59:09	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:08.451	NULL	NULL	-51	NULL	747553882	-1930467250	-51.0	NULL	TRUE	0	1969-12-31 16:00:08.451	q8M86Fx0r	q8M86Fx0r	q8M86Fx0r	-51.0	1.495107764E9	-0.6702291758433747	7.47553883E8
+true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-335450417	1233327000	11	NULL	1	2	NULL	NULL	11	11	11	11.0	NULL	-3.35450417E8	1.233327E9	11.0	NULL	1.0	2.351	NULL	NULL	-3.35450432E8	NULL	1969-12-31 16:00:00.011	NULL	1969-12-27 18:49:09.583	1970-01-14 22:35:27	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-335450417	1233327000	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	dOYnqgaXoJ1P3ERwxe5N7	dOYnqgaXoJ	dOYnqgaXoJ	11.0	-6.70900834E8	-0.9999902065507035	-3.35450416E8
+true	NULL	true	true	true	NULL	true	false	true	true	11	NULL	-64615982	1803053750	11	NULL	1	2	NULL	8	11	11	11	11.0	NULL	-6.4615982E7	1.80305375E9	11.0	NULL	1.0	2.351	NULL	8.0	-6.4615984E7	NULL	1969-12-31 16:00:00.011	NULL	1969-12-30 22:03:04.018	1970-01-21 12:50:53.75	1969-12-31 16:00:11	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:02.351	NULL	NULL	11	NULL	-64615982	1803053750	11.0	NULL	TRUE	0	1969-12-31 16:00:02.351	8J5OB7K26PEV7kdbeHr3	8J5OB7K26P	8J5OB7K26P	11.0	-1.29231964E8	-0.9999902065507035	-6.4615981E7
+true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	890988972	-1862301000	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	8.90988972E8	-1.862301E9	8.0	NULL	1.0	15.892	NULL	NULL	8.9098899E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-10 23:29:48.972	1969-12-10 02:41:39	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	890988972	-1862301000	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	XylAH4	XylAH4	XylAH4	8.0	1.781977944E9	0.9893582466233818	8.90988973E8
+true	NULL	true	true	true	NULL	true	false	true	true	8	NULL	930867246	1205399250	8	NULL	1	15	NULL	NULL	8	8	8	8.0	NULL	9.30867246E8	1.20539925E9	8.0	NULL	1.0	15.892	NULL	NULL	9.3086726E8	NULL	1969-12-31 16:00:00.008	NULL	1970-01-11 10:34:27.246	1970-01-14 14:49:59.25	1969-12-31 16:00:08	NULL	1969-12-31 16:00:00.001	1969-12-31 16:00:00	1969-12-31 16:00:15.892	NULL	NULL	8	NULL	930867246	1205399250	8.0	NULL	TRUE	0	1969-12-31 16:00:15.892	c1V8o1A	c1V8o1A	c1V8o1A	8.0	1.861734492E9	0.9893582466233818	9.30867247E8
+true	true	NULL	true	true	true	NULL	false	true	NULL	-14	-7196	NULL	-1552199500	-14	-7196	NULL	11	NULL	NULL	-14	-14	-14	-14.0	-7196.0	NULL	-1.5521995E9	-14.0	-7196.0	NULL	11.065	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.986	1969-12-31 15:59:52.804	NULL	1969-12-13 16:50:00.5	1969-12-31 15:59:46	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:11.065	NULL	NULL	-14	-7196	NULL	-1552199500	-14.0	-7196.0	NULL	0	1969-12-31 16:00:11.065	NULL	NULL	NULL	-14.0	NULL	-0.9906073556948704	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-21	-7196	NULL	1542429000	-21	-7196	NULL	-4	NULL	NULL	-21	-21	-21	-21.0	-7196.0	NULL	1.542429E9	-21.0	-7196.0	NULL	-4.1	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.979	1969-12-31 15:59:52.804	NULL	1970-01-18 12:27:09	1969-12-31 15:59:39	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:55.9	NULL	NULL	-21	-7196	NULL	1542429000	-21.0	-7196.0	NULL	0	1969-12-31 15:59:55.9	NULL	NULL	NULL	-21.0	NULL	-0.8366556385360561	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-24	-7196	NULL	829111000	-24	-7196	NULL	-6	NULL	NULL	-24	-24	-24	-24.0	-7196.0	NULL	8.29111E8	-24.0	-7196.0	NULL	-6.855	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.976	1969-12-31 15:59:52.804	NULL	1970-01-10 06:18:31	1969-12-31 15:59:36	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.145	NULL	NULL	-24	-7196	NULL	829111000	-24.0	-7196.0	NULL	0	1969-12-31 15:59:53.145	NULL	NULL	NULL	-24.0	NULL	0.9055783620066238	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-30	-200	NULL	1429852250	-30	-200	NULL	12	NULL	NULL	-30	-30	-30	-30.0	-200.0	NULL	1.42985225E9	-30.0	-200.0	NULL	12.935	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.97	1969-12-31 15:59:59.8	NULL	1970-01-17 05:10:52.25	1969-12-31 15:59:30	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 16:00:12.935	NULL	NULL	-30	-200	NULL	1429852250	-30.0	-200.0	NULL	0	1969-12-31 16:00:12.935	NULL	NULL	NULL	-30.0	NULL	0.9880316240928618	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	-2006216750	-36	-200	NULL	-14	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	-2.00621675E9	-36.0	-200.0	NULL	-14.252	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1969-12-08 10:43:03.25	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.748	NULL	NULL	-36	-200	NULL	-2006216750	-36.0	-200.0	NULL	0	1969-12-31 15:59:45.748	NULL	NULL	NULL	-36.0	NULL	0.9917788534431158	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-36	-200	NULL	1599879000	-36	-200	NULL	-6	NULL	NULL	-36	-36	-36	-36.0	-200.0	NULL	1.599879E9	-36.0	-200.0	NULL	-6.183	NULL	NULL	NULL	-200.0	1969-12-31 15:59:59.964	1969-12-31 15:59:59.8	NULL	1970-01-19 04:24:39	1969-12-31 15:59:24	1969-12-31 15:56:40	NULL	1969-12-31 16:00:00	1969-12-31 15:59:53.817	NULL	NULL	-36	-200	NULL	1599879000	-36.0	-200.0	NULL	0	1969-12-31 15:59:53.817	NULL	NULL	NULL	-36.0	NULL	0.9917788534431158	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-38	15601	NULL	-1858689000	-38	15601	NULL	-1	NULL	NULL	-38	-38	-38	-38.0	15601.0	NULL	-1.858689E9	-38.0	15601.0	NULL	-1.386	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.962	1969-12-31 16:00:15.601	NULL	1969-12-10 03:41:51	1969-12-31 15:59:22	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:58.614	NULL	NULL	-38	15601	NULL	-1858689000	-38.0	15601.0	NULL	0	1969-12-31 15:59:58.614	NULL	NULL	NULL	-38.0	NULL	-0.2963685787093853	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-5	15601	NULL	612416000	-5	15601	NULL	4	NULL	NULL	-5	-5	-5	-5.0	15601.0	NULL	6.12416E8	-5.0	15601.0	NULL	4.679	NULL	NULL	NULL	15601.0	1969-12-31 15:59:59.995	1969-12-31 16:00:15.601	NULL	1970-01-07 18:06:56	1969-12-31 15:59:55	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 16:00:04.679	NULL	NULL	-5	15601	NULL	612416000	-5.0	15601.0	NULL	0	1969-12-31 16:00:04.679	NULL	NULL	NULL	-5.0	NULL	0.9589242746631385	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-50	-7196	NULL	-1031187250	-50	-7196	NULL	-5	NULL	NULL	-50	-50	-50	-50.0	-7196.0	NULL	-1.03118725E9	-50.0	-7196.0	NULL	-5.267	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.95	1969-12-31 15:59:52.804	NULL	1969-12-19 17:33:32.75	1969-12-31 15:59:10	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:54.733	NULL	NULL	-50	-7196	NULL	-1031187250	-50.0	-7196.0	NULL	0	1969-12-31 15:59:54.733	NULL	NULL	NULL	-50.0	NULL	0.26237485370392877	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-59	-7196	NULL	-1604890000	-59	-7196	NULL	13	NULL	NULL	-59	-59	-59	-59.0	-7196.0	NULL	-1.60489E9	-59.0	-7196.0	NULL	13.15	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.941	1969-12-31 15:59:52.804	NULL	1969-12-13 02:11:50	1969-12-31 15:59:01	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:13.15	NULL	NULL	-59	-7196	NULL	-1604890000	-59.0	-7196.0	NULL	0	1969-12-31 16:00:13.15	NULL	NULL	NULL	-59.0	NULL	-0.6367380071391379	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-60	-7196	NULL	1516314750	-60	-7196	NULL	-7	NULL	NULL	-60	-60	-60	-60.0	-7196.0	NULL	1.51631475E9	-60.0	-7196.0	NULL	-7.592	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.94	1969-12-31 15:59:52.804	NULL	1970-01-18 05:11:54.75	1969-12-31 15:59:00	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 15:59:52.408	NULL	NULL	-60	-7196	NULL	1516314750	-60.0	-7196.0	NULL	0	1969-12-31 15:59:52.408	NULL	NULL	NULL	-60.0	NULL	0.3048106211022167	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	-8	-7196	NULL	-1849991500	-8	-7196	NULL	3	NULL	NULL	-8	-8	-8	-8.0	-7196.0	NULL	-1.8499915E9	-8.0	-7196.0	NULL	3.136	NULL	NULL	NULL	-7196.0	1969-12-31 15:59:59.992	1969-12-31 15:59:52.804	NULL	1969-12-10 06:06:48.5	1969-12-31 15:59:52	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:03.136	NULL	NULL	-8	-7196	NULL	-1849991500	-8.0	-7196.0	NULL	0	1969-12-31 16:00:03.136	NULL	NULL	NULL	-8.0	NULL	-0.9893582466233818	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	20	15601	NULL	-362433250	20	15601	NULL	-14	NULL	NULL	20	20	20	20.0	15601.0	NULL	-3.6243325E8	20.0	15601.0	NULL	-14.871	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.02	1969-12-31 16:00:15.601	NULL	1969-12-27 11:19:26.75	1969-12-31 16:00:20	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:45.129	NULL	NULL	20	15601	NULL	-362433250	20.0	15601.0	NULL	0	1969-12-31 15:59:45.129	NULL	NULL	NULL	20.0	NULL	0.9129452507276277	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	48	15601	NULL	-795361000	48	15601	NULL	-9	NULL	NULL	48	48	48	48.0	15601.0	NULL	-7.95361E8	48.0	15601.0	NULL	-9.765	NULL	NULL	NULL	15601.0	1969-12-31 16:00:00.048	1969-12-31 16:00:15.601	NULL	1969-12-22 11:03:59	1969-12-31 16:00:48	1969-12-31 20:20:01	NULL	1969-12-31 16:00:00	1969-12-31 15:59:50.235	NULL	NULL	48	15601	NULL	-795361000	48.0	15601.0	NULL	0	1969-12-31 15:59:50.235	NULL	NULL	NULL	48.0	NULL	-0.7682546613236668	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	5	-7196	NULL	-1015607500	5	-7196	NULL	10	NULL	NULL	5	5	5	5.0	-7196.0	NULL	-1.0156075E9	5.0	-7196.0	NULL	10.973	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.005	1969-12-31 15:59:52.804	NULL	1969-12-19 21:53:12.5	1969-12-31 16:00:05	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.973	NULL	NULL	5	-7196	NULL	-1015607500	5.0	-7196.0	NULL	0	1969-12-31 16:00:10.973	NULL	NULL	NULL	5.0	NULL	-0.9589242746631385	NULL
+true	true	NULL	true	true	true	NULL	false	true	NULL	59	-7196	NULL	-1137754500	59	-7196	NULL	10	NULL	NULL	59	59	59	59.0	-7196.0	NULL	-1.1377545E9	59.0	-7196.0	NULL	10.956	NULL	NULL	NULL	-7196.0	1969-12-31 16:00:00.059	1969-12-31 15:59:52.804	NULL	1969-12-18 11:57:25.5	1969-12-31 16:00:59	1969-12-31 14:00:04	NULL	1969-12-31 16:00:00	1969-12-31 16:00:10.956	NULL	NULL	59	-7196	NULL	-1137754500	59.0	-7196.0	NULL	0	1969-12-31 16:00:10.956	NULL	NULL	NULL	59.0	NULL	0.6367380071391379	NULL