You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2017/11/17 13:48:16 UTC

[2/2] hive git commit: HIVE-18077: Vectorization: Add string conversion case for UDFToDouble (Matt McCline, reviewed by Teddy Choi)

HIVE-18077: Vectorization: Add string conversion case for UDFToDouble (Matt McCline, reviewed by Teddy Choi)


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

Branch: refs/heads/master
Commit: c58435bf8bdd29fc3544bae27d7965d8665dff38
Parents: 987d130
Author: Matt McCline <mm...@hortonworks.com>
Authored: Fri Nov 17 07:48:05 2017 -0600
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Fri Nov 17 07:48:05 2017 -0600

----------------------------------------------------------------------
 .../ql/exec/vector/VectorizationContext.java    |  12 ++
 .../vector/expressions/CastDoubleToChar.java    |  54 +++++++
 .../vector/expressions/CastDoubleToString.java  |  46 ++++++
 .../vector/expressions/CastDoubleToVarChar.java |  54 +++++++
 .../vector/expressions/CastFloatToChar.java     |  54 +++++++
 .../vector/expressions/CastFloatToString.java   |  47 ++++++
 .../vector/expressions/CastFloatToVarChar.java  |  54 +++++++
 .../vector/expressions/CastStringToDouble.java  | 159 +++++++++++++++++++
 .../vector/expressions/CastStringToFloat.java   |  72 +++++++++
 .../expressions/DoubleToStringUnaryUDF.java     | 133 ++++++++++++++++
 .../apache/hadoop/hive/ql/udf/UDFToDouble.java  |   3 +-
 .../apache/hadoop/hive/ql/udf/UDFToFloat.java   |   3 +-
 .../clientpositive/llap/vector_char_4.q.out     |   4 +-
 .../clientpositive/llap/vector_coalesce.q.out   |   4 +-
 .../llap/vector_groupby_grouping_sets1.q.out    |   4 +-
 .../llap/vector_groupby_grouping_sets4.q.out    |  12 +-
 .../llap/vector_groupby_grouping_sets6.q.out    |   8 +-
 .../vector_groupby_grouping_sets_limit.q.out    |   4 +-
 .../clientpositive/llap/vector_varchar_4.q.out  |   4 +-
 .../clientpositive/llap/vectorized_casts.q.out  |   2 +-
 .../vectorized_dynamic_partition_pruning.q.out  |  56 +++----
 ...k_vectorized_dynamic_partition_pruning.q.out | 144 ++++++++---------
 .../clientpositive/spark/vector_char_4.q.out    |   4 +-
 .../clientpositive/spark/vector_varchar_4.q.out |   4 +-
 .../results/clientpositive/vector_char_4.q.out  |   4 +-
 .../clientpositive/vector_coalesce.q.out        |   4 +-
 .../clientpositive/vector_varchar_4.q.out       |   4 +-
 .../clientpositive/vectorized_casts.q.out       |   2 +-
 28 files changed, 821 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/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 84dcb17..13eff51 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
@@ -2604,6 +2604,10 @@ public class VectorizationContext {
       return createVectorExpression(CastBooleanToStringViaLongToString.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isIntFamily(inputType)) {
       return createVectorExpression(CastLongToString.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+    } else if (inputType.equals("float")) {
+      return createVectorExpression(CastFloatToString.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+    } else if (inputType.equals("double")) {
+      return createVectorExpression(CastDoubleToString.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isDecimalFamily(inputType)) {
       return createVectorExpression(CastDecimalToString.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isDateFamily(inputType)) {
@@ -2628,6 +2632,10 @@ public class VectorizationContext {
       return createVectorExpression(CastBooleanToCharViaLongToChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isIntFamily(inputType)) {
       return createVectorExpression(CastLongToChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+    } else if (inputType.equals("float")) {
+      return createVectorExpression(CastFloatToChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+    } else if (inputType.equals("double")) {
+      return createVectorExpression(CastDoubleToChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isDecimalFamily(inputType)) {
       return createVectorExpression(CastDecimalToChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isDateFamily(inputType)) {
@@ -2652,6 +2660,10 @@ public class VectorizationContext {
       return createVectorExpression(CastBooleanToVarCharViaLongToVarChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isIntFamily(inputType)) {
       return createVectorExpression(CastLongToVarChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+    } else if (inputType.equals("float")) {
+      return createVectorExpression(CastFloatToVarChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+    } else if (inputType.equals("double")) {
+      return createVectorExpression(CastDoubleToVarChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isDecimalFamily(inputType)) {
       return createVectorExpression(CastDecimalToVarChar.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
     } else if (isDateFamily(inputType)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToChar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToChar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToChar.java
new file mode 100644
index 0000000..e9eb46a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToChar.java
@@ -0,0 +1,54 @@
+/**
+ * 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;
+
+public class CastDoubleToChar extends CastDoubleToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastDoubleToChar() {
+    super();
+  }
+
+  public CastDoubleToChar(int inputColumn, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.rightTrimAndTruncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, inputColumn) + ", maxLength " + maxLength;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToString.java
new file mode 100644
index 0000000..b099a7f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToString.java
@@ -0,0 +1,46 @@
+/**
+ * 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;
+import java.nio.charset.StandardCharsets;
+
+public class CastDoubleToString extends DoubleToStringUnaryUDF {
+  private static final long serialVersionUID = 1L;
+
+  public CastDoubleToString() {
+    super();
+  }
+
+  public CastDoubleToString(int inputColumn, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+  }
+
+  // The assign method will be overridden for CHAR and VARCHAR.
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    outV.setVal(i, bytes, 0, length);
+  }
+
+  @Override
+  protected void func(BytesColumnVector outV, double[] vector, int i) {
+    String string = String.valueOf(vector[i]);
+    byte[] bytes = string.getBytes(StandardCharsets.UTF_8);
+    assign(outV, i, bytes, bytes.length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToVarChar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToVarChar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToVarChar.java
new file mode 100644
index 0000000..6dd8578
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToVarChar.java
@@ -0,0 +1,54 @@
+/**
+ * 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;
+
+public class CastDoubleToVarChar extends CastDoubleToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastDoubleToVarChar() {
+    super();
+  }
+
+  public CastDoubleToVarChar(int inputColumn, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.truncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, inputColumn) + ", maxLength " + maxLength;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToChar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToChar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToChar.java
new file mode 100644
index 0000000..b2ce640
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToChar.java
@@ -0,0 +1,54 @@
+/**
+ * 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;
+
+public class CastFloatToChar extends CastFloatToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastFloatToChar() {
+    super();
+  }
+
+  public CastFloatToChar(int inputColumn, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.rightTrimAndTruncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, inputColumn) + ", maxLength " + maxLength;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToString.java
new file mode 100644
index 0000000..5e94502
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToString.java
@@ -0,0 +1,47 @@
+/**
+ * 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 java.nio.charset.StandardCharsets;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+public class CastFloatToString extends DoubleToStringUnaryUDF {
+  private static final long serialVersionUID = 1L;
+
+  public CastFloatToString() {
+    super();
+  }
+
+  public CastFloatToString(int inputColumn, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+  }
+
+  // The assign method will be overridden for CHAR and VARCHAR.
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    outV.setVal(i, bytes, 0, length);
+  }
+
+  @Override
+  protected void func(BytesColumnVector outV, double[] vector, int i) {
+    String string = String.valueOf((float) vector[i]);
+    byte[] bytes = string.getBytes(StandardCharsets.UTF_8);
+    assign(outV, i, bytes, bytes.length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToVarChar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToVarChar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToVarChar.java
new file mode 100644
index 0000000..956cb42
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastFloatToVarChar.java
@@ -0,0 +1,54 @@
+/**
+ * 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;
+
+public class CastFloatToVarChar extends CastFloatToString implements TruncStringOutput {
+  private static final long serialVersionUID = 1L;
+  private int maxLength; // Must be manually set with setMaxLength.
+
+  public CastFloatToVarChar() {
+    super();
+  }
+
+  public CastFloatToVarChar(int inputColumn, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+  }
+
+  @Override
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
+    StringExpr.truncate(outV, i, bytes, 0, length, maxLength);
+  }
+
+  @Override
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  @Override
+  public void setMaxLength(int maxLength) {
+    this.maxLength = maxLength;
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, inputColumn) + ", maxLength " + maxLength;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java
new file mode 100644
index 0000000..52a0c2e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDouble.java
@@ -0,0 +1,159 @@
+/**
+ * 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;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.serde2.lazy.LazyUtils;
+import org.apache.hadoop.hive.serde2.lazy.fast.StringToDouble;
+
+/**
+ * Cast a string to a double.
+ *
+ * If other functions besides cast need to take a string in and produce a long,
+ * you can subclass this class or convert it to a superclass, and
+ * implement different "func()" methods for each operation.
+ */
+public class CastStringToDouble extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+  int inputColumn;
+
+  public CastStringToDouble(int inputColumn, int outputColumnNum) {
+    super(outputColumnNum);
+    this.inputColumn = inputColumn;
+  }
+
+  public CastStringToDouble() {
+    super();
+
+    // Dummy final assignments.
+    inputColumn = -1;
+  }
+
+  /**
+   * Convert input string to a double, at position i in the respective vectors.
+   */
+  protected void func(DoubleColumnVector outV, BytesColumnVector inV, int batchIndex) {
+
+    byte[] bytes = inV.vector[batchIndex];
+    final int start = inV.start[batchIndex];
+    final int length = inV.length[batchIndex];
+    try {
+      if (!LazyUtils.isNumberMaybe(bytes, start, length)) {
+        outV.noNulls = false;
+        outV.isNull[batchIndex] = true;
+        outV.vector[batchIndex] = DoubleColumnVector.NULL_VALUE;
+        return;
+      }
+      outV.vector[batchIndex] = StringToDouble.strtod(bytes, start, length);
+    } catch (Exception e) {
+
+      // for any exception in conversion to integer, produce NULL
+      outV.noNulls = false;
+      outV.isNull[batchIndex] = true;
+      outV.vector[batchIndex] = DoubleColumnVector.NULL_VALUE;
+    }
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    BytesColumnVector inV = (BytesColumnVector) batch.cols[inputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    DoubleColumnVector outV = (DoubleColumnVector) batch.cols[outputColumnNum];
+
+    if (n == 0) {
+
+      // Nothing to do
+      return;
+    }
+
+    if (inV.noNulls) {
+      outV.noNulls = true;
+      if (inV.isRepeating) {
+        outV.isRepeating = true;
+        func(outV, inV, 0);
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          func(outV, inV, i);
+        }
+        outV.isRepeating = false;
+      } else {
+        for(int i = 0; i != n; i++) {
+          func(outV, inV, i);
+        }
+        outV.isRepeating = false;
+      }
+    } else {
+
+      // Handle case with nulls. Don't do function if the value is null,
+      // because the data may be undefined for a null value.
+      outV.noNulls = false;
+      if (inV.isRepeating) {
+        outV.isRepeating = true;
+        outV.isNull[0] = inV.isNull[0];
+        if (!inV.isNull[0]) {
+          func(outV, inV, 0);
+        }
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outV.isNull[i] = inV.isNull[i];
+          if (!inV.isNull[i]) {
+            func(outV, inV, i);
+          }
+        }
+        outV.isRepeating = false;
+      } else {
+        System.arraycopy(inV.isNull, 0, outV.isNull, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!inV.isNull[i]) {
+            func(outV, inV, i);
+          }
+        }
+        outV.isRepeating = false;
+      }
+    }
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, inputColumn);
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    VectorExpressionDescriptor.Builder b = new VectorExpressionDescriptor.Builder();
+    b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(1)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN);
+    return b.build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToFloat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToFloat.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToFloat.java
new file mode 100644
index 0000000..1f79272
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToFloat.java
@@ -0,0 +1,72 @@
+/**
+ * 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 java.nio.charset.StandardCharsets;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.serde2.lazy.LazyUtils;
+
+/**
+ * Cast a string to a double.
+ *
+ * If other functions besides cast need to take a string in and produce a long,
+ * you can subclass this class or convert it to a superclass, and
+ * implement different "func()" methods for each operation.
+ */
+public class CastStringToFloat extends CastStringToDouble {
+  private static final long serialVersionUID = 1L;
+
+  public CastStringToFloat(int inputColumn, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+  }
+
+  public CastStringToFloat() {
+    super();
+  }
+
+  /**
+   * Convert input string to a double, at position i in the respective vectors.
+   */
+  @Override
+  protected void func(DoubleColumnVector outV, BytesColumnVector inV, int batchIndex) {
+
+    byte[] bytes = inV.vector[batchIndex];
+    final int start = inV.start[batchIndex];
+    final int length = inV.length[batchIndex];
+    try {
+      if (!LazyUtils.isNumberMaybe(bytes, start, length)) {
+        outV.noNulls = false;
+        outV.isNull[batchIndex] = true;
+        outV.vector[batchIndex] = DoubleColumnVector.NULL_VALUE;
+        return;
+      }
+      outV.vector[batchIndex] =
+          Float.parseFloat(
+              new String(bytes, start, length, StandardCharsets.UTF_8));
+    } catch (Exception e) {
+
+      // for any exception in conversion to integer, produce NULL
+      outV.noNulls = false;
+      outV.isNull[batchIndex] = true;
+      outV.vector[batchIndex] = DoubleColumnVector.NULL_VALUE;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java
new file mode 100644
index 0000000..536e797
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DoubleToStringUnaryUDF.java
@@ -0,0 +1,133 @@
+/**
+ * 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;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a superclass for unary double functions returning strings that operate directly on the
+ * input and set the output.
+ */
+abstract public class DoubleToStringUnaryUDF extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+
+  protected final int inputColumn;
+
+  public DoubleToStringUnaryUDF(int inputColumn, int outputColumnNum) {
+    super(outputColumnNum);
+    this.inputColumn = inputColumn;
+  }
+
+  public DoubleToStringUnaryUDF() {
+    super();
+
+    // Dummy final assignments.
+    inputColumn = -1;
+  }
+
+  abstract protected void func(BytesColumnVector outV, double[] vector, int i);
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[inputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    BytesColumnVector outV = (BytesColumnVector) batch.cols[outputColumnNum];
+    outV.initBuffer();
+
+    if (n == 0) {
+      //Nothing to do
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      outV.noNulls = true;
+      if (inputColVector.isRepeating) {
+        outV.isRepeating = true;
+        func(outV, vector, 0);
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          func(outV, vector, i);
+        }
+        outV.isRepeating = false;
+      } else {
+        for(int i = 0; i != n; i++) {
+          func(outV, vector, i);
+        }
+        outV.isRepeating = false;
+      }
+    } else {
+
+      // Handle case with nulls. Don't do function if the value is null,
+      // because the data may be undefined for a null value.
+      outV.noNulls = false;
+      if (inputColVector.isRepeating) {
+        outV.isRepeating = true;
+        outV.isNull[0] = inputColVector.isNull[0];
+        if (!inputColVector.isNull[0]) {
+          func(outV, vector, 0);
+        }
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outV.isNull[i] = inputColVector.isNull[i];
+          if (!inputColVector.isNull[i]) {
+            func(outV, vector, i);
+          }
+        }
+        outV.isRepeating = false;
+      } else {
+        System.arraycopy(inputColVector.isNull, 0, outV.isNull, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!inputColVector.isNull[i]) {
+            func(outV, vector, i);
+          }
+        }
+        outV.isRepeating = false;
+      }
+    }
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, inputColumn);
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    VectorExpressionDescriptor.Builder b = new VectorExpressionDescriptor.Builder();
+    b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(1)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.INT_FAMILY)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN);
+    return b.build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java
index 5fcae42..b08c3cc 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.udf;
 import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDecimalToDouble;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.CastLongToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToDouble;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
@@ -40,7 +41,7 @@ import org.apache.hadoop.io.Text;
  *
  */
 @VectorizedExpressions({CastTimestampToDouble.class, CastLongToDouble.class,
-    CastDecimalToDouble.class})
+    CastDecimalToDouble.class, CastStringToDouble.class})
 public class UDFToDouble extends UDF {
   private final DoubleWritable doubleWritable = new DoubleWritable();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
index c8e32f4..8debe2e 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.udf;
 import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDecimalToDouble;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToFloat;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.CastLongToFloatViaLongToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastTimestampToDouble;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
@@ -41,7 +42,7 @@ import org.apache.hadoop.io.Text;
  *
  */
 @VectorizedExpressions({CastTimestampToDouble.class, CastLongToFloatViaLongToDouble.class,
-    CastDecimalToDouble.class})
+    CastDecimalToDouble.class, CastStringToFloat.class})
 public class UDFToFloat extends UDF {
   private final FloatWritable floatWritable = new FloatWritable();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vector_char_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_char_4.q.out b/ql/src/test/results/clientpositive/llap/vector_char_4.q.out
index a459f8b..ba704350 100644
--- a/ql/src/test/results/clientpositive/llap/vector_char_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_char_4.q.out
@@ -156,7 +156,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [14, 15, 16, 17, 18, 19, 20]
-                        selectExpressions: CastLongToChar(col 0:tinyint, maxLength 10) -> 14:char(10), CastLongToChar(col 1:smallint, maxLength 10) -> 15:char(10), CastLongToChar(col 2:int, maxLength 20) -> 16:char(20), CastLongToChar(col 3:bigint, maxLength 30) -> 17:char(30), VectorUDFAdaptor(CAST( f AS CHAR(20)) -> 18:char(20), VectorUDFAdaptor(CAST( d AS CHAR(20)) -> 19:char(20), CastStringGroupToChar(col 8:string, maxLength 50) -> 20:char(50)
+                        selectExpressions: CastLongToChar(col 0:tinyint, maxLength 10) -> 14:char(10), CastLongToChar(col 1:smallint, maxLength 10) -> 15:char(10), CastLongToChar(col 2:int, maxLength 20) -> 16:char(20), CastLongToChar(col 3:bigint, maxLength 30) -> 17:char(30), CastFloatToChar(col 4:float, maxLength 20) -> 18:char(20), CastDoubleToChar(col 5:double, maxLength 20) -> 19:char(20), CastStringGroupToChar(col 8:string, maxLength 50) -> 20:char(50)
                     Statistics: Num rows: 2000 Data size: 410616 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -178,7 +178,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
 
   Stage: Stage-2

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out b/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
index 0ebcb84..339df62 100644
--- a/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
@@ -36,7 +36,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [6, 2, 4, 1, 17]
-                          selectExpressions: VectorCoalesce(columns [13, 6, 14, 15, 16])(children: ConstantVectorExpression(val null) -> 13:string, col 6:string, CastLongToString(col 2:int) -> 14:string, VectorUDFAdaptor(null(cfloat)) -> 15:string, CastLongToString(col 1:smallint) -> 16:string) -> 17:string
+                          selectExpressions: VectorCoalesce(columns [13, 6, 14, 15, 16])(children: ConstantVectorExpression(val null) -> 13:string, col 6:string, CastLongToString(col 2:int) -> 14:string, CastFloatToString(col 4:float) -> 15:string, CastLongToString(col 1:smallint) -> 16:string) -> 17:string
                         Reduce Sink Vectorization:
                             className: VectorReduceSinkObjectHashOperator
                             native: true
@@ -50,7 +50,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets1.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets1.q.out
index e1c5fd8..d2b738b 100644
--- a/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets1.q.out
@@ -1019,7 +1019,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [6]
-                        selectExpressions: DoubleColAddDoubleColumn(col 4:double, col 5:double)(children: VectorUDFAdaptor(UDFToDouble(a)) -> 4:double, VectorUDFAdaptor(UDFToDouble(b)) -> 5:double) -> 6:double
+                        selectExpressions: DoubleColAddDoubleColumn(col 4:double, col 5:double)(children: CastStringToDouble(col 0:string) -> 4:double, CastStringToDouble(col 1:string) -> 5:double) -> 6:double
                     Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
@@ -1056,7 +1056,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets4.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets4.q.out
index 09d119d..957bc22 100644
--- a/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets4.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets4.q.out
@@ -73,7 +73,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterDoubleColLessDoubleScalar(col 4:double, val 3.0)(children: VectorUDFAdaptor(UDFToDouble(a)) -> 4:double)
+                        predicateExpression: FilterDoubleColLessDoubleScalar(col 4:double, val 3.0)(children: CastStringToDouble(col 0:string) -> 4:double)
                     predicate: (UDFToDouble(a) < 3.0) (type: boolean)
                     Statistics: Num rows: 2 Data size: 736 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
@@ -123,7 +123,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3
@@ -317,7 +317,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterDoubleColLessDoubleScalar(col 4:double, val 3.0)(children: VectorUDFAdaptor(UDFToDouble(a)) -> 4:double)
+                        predicateExpression: FilterDoubleColLessDoubleScalar(col 4:double, val 3.0)(children: CastStringToDouble(col 0:string) -> 4:double)
                     predicate: (UDFToDouble(a) < 3.0) (type: boolean)
                     Statistics: Num rows: 2 Data size: 736 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
@@ -367,7 +367,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3
@@ -593,7 +593,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterDoubleColLessDoubleScalar(col 4:double, val 3.0)(children: VectorUDFAdaptor(UDFToDouble(a)) -> 4:double)
+                        predicateExpression: FilterDoubleColLessDoubleScalar(col 4:double, val 3.0)(children: CastStringToDouble(col 0:string) -> 4:double)
                     predicate: (UDFToDouble(a) < 3.0) (type: boolean)
                     Statistics: Num rows: 2 Data size: 736 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
@@ -643,7 +643,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets6.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets6.q.out
index 79be2be..5e9e204 100644
--- a/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets6.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets6.q.out
@@ -67,7 +67,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterDoubleColEqualDoubleScalar(col 4:double, val 5.0)(children: VectorUDFAdaptor(UDFToDouble(a)) -> 4:double)
+                        predicateExpression: FilterDoubleColEqualDoubleScalar(col 4:double, val 5.0)(children: CastStringToDouble(col 0:string) -> 4:double)
                     predicate: (UDFToDouble(a) = 5.0) (type: boolean)
                     Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
@@ -102,7 +102,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3
@@ -210,7 +210,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterDoubleColEqualDoubleScalar(col 4:double, val 5.0)(children: VectorUDFAdaptor(UDFToDouble(a)) -> 4:double)
+                        predicateExpression: FilterDoubleColEqualDoubleScalar(col 4:double, val 5.0)(children: CastStringToDouble(col 0:string) -> 4:double)
                     predicate: (UDFToDouble(a) = 5.0) (type: boolean)
                     Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
@@ -245,7 +245,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets_limit.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets_limit.q.out b/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets_limit.q.out
index c135a46..e8ca06e 100644
--- a/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets_limit.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_groupby_grouping_sets_limit.q.out
@@ -1046,7 +1046,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [6]
-                        selectExpressions: DoubleColAddDoubleColumn(col 4:double, col 5:double)(children: VectorUDFAdaptor(UDFToDouble(a)) -> 4:double, VectorUDFAdaptor(UDFToDouble(b)) -> 5:double) -> 6:double
+                        selectExpressions: DoubleColAddDoubleColumn(col 4:double, col 5:double)(children: CastStringToDouble(col 0:string) -> 4:double, CastStringToDouble(col 1:string) -> 5:double) -> 6:double
                     Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
@@ -1084,7 +1084,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vector_varchar_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_varchar_4.q.out b/ql/src/test/results/clientpositive/llap/vector_varchar_4.q.out
index d883c6d..c04cc61 100644
--- a/ql/src/test/results/clientpositive/llap/vector_varchar_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_varchar_4.q.out
@@ -156,7 +156,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [14, 15, 16, 17, 18, 19, 20]
-                        selectExpressions: CastLongToVarChar(col 0:tinyint, maxLength 10) -> 14:varchar(10), CastLongToVarChar(col 1:smallint, maxLength 10) -> 15:varchar(10), CastLongToVarChar(col 2:int, maxLength 20) -> 16:varchar(20), CastLongToVarChar(col 3:bigint, maxLength 30) -> 17:varchar(30), VectorUDFAdaptor(CAST( f AS varchar(20))) -> 18:varchar(20), VectorUDFAdaptor(CAST( d AS varchar(20))) -> 19:varchar(20), CastStringGroupToVarChar(col 8:string, maxLength 50) -> 20:varchar(50)
+                        selectExpressions: CastLongToVarChar(col 0:tinyint, maxLength 10) -> 14:varchar(10), CastLongToVarChar(col 1:smallint, maxLength 10) -> 15:varchar(10), CastLongToVarChar(col 2:int, maxLength 20) -> 16:varchar(20), CastLongToVarChar(col 3:bigint, maxLength 30) -> 17:varchar(30), CastFloatToVarChar(col 4:float, maxLength 20) -> 18:varchar(20), CastDoubleToVarChar(col 5:double, maxLength 20) -> 19:varchar(20), CastStringGroupToVarChar(col 8:string, maxLength 50) -> 20:varchar(50)
                     Statistics: Num rows: 2000 Data size: 410616 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -178,7 +178,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
 
   Stage: Stage-2

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out b/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
index 09fd402..ad6d97b 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
@@ -183,7 +183,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [13, 14, 15, 16, 17, 18, 10, 20, 19, 22, 0, 1, 2, 3, 21, 23, 10, 24, 25, 27, 28, 29, 30, 31, 32, 33, 34, 4, 5, 35, 36, 37, 38, 39, 5, 41, 43, 45, 47, 48, 49, 51, 54, 55, 8, 56, 57, 26, 58, 59, 60, 61, 62, 63, 64, 65, 6, 67, 68, 69, 70, 66, 73]
-                          selectExpressions: CastLongToBooleanViaLongToLong(col 0:tinyint) -> 13:boolean, CastLongToBooleanViaLongToLong(col 1:smallint) -> 14:boolean, CastLongToBooleanViaLongToLong(col 2:int) -> 15:boolean, CastLongToBooleanViaLongToLong(col 3:bigint) -> 16:boolean, CastDoubleToBooleanViaDoubleToLong(col 4:float) -> 17:boolean, CastDoubleToBooleanViaDoubleToLong(col 5:double) -> 18:boolean, CastLongToBooleanViaLongToLong(col 19:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 19:bigint) -> 20:boolean, CastTimestampToBoolean(col 8:timestamp) -> 19:boolean, CastLongToBooleanViaLongToLong(col 21:bigint)(children: StringLength(col 6:string) -> 21:bigint) -> 22:boolean, CastDoubleToLong(col 4:float) -> 21:int, CastDoubleToLong(col 5:double) -> 23:int, CastTimestampToLong(col 8:timestamp) -> 24:int, CastStringToLong(col 6:string) -> 25:int, CastStringToLong(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) 
 -> 27:int, CastDoubleToLong(col 4:float) -> 28:tinyint, CastDoubleToLong(col 4:float) -> 29:smallint, CastDoubleToLong(col 4:float) -> 30:bigint, CastLongToDouble(col 0:tinyint) -> 31:double, CastLongToDouble(col 1:smallint) -> 32:double, CastLongToDouble(col 2:int) -> 33:double, CastLongToDouble(col 3:bigint) -> 34:double, CastLongToDouble(col 10:boolean) -> 35:double, CastTimestampToDouble(col 8:timestamp) -> 36:double, VectorUDFAdaptor(UDFToDouble(cstring1)) -> 37:double, VectorUDFAdaptor(UDFToDouble(substr(cstring1, 1, 1)))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 38:double, CastLongToFloatViaLongToDouble(col 2:int) -> 39:float, CastMillisecondsLongToTimestamp(col 0:tinyint) -> 41:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 43:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 45:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 47:timestamp, CastDoubleToTimestamp(col 4:float) -> 48:timestamp, CastDoubl
 eToTimestamp(col 5:double) -> 49:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 51:timestamp, CastMillisecondsLongToTimestamp(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 54:timestamp, CastDateToTimestamp(col 52:date)(children: CastTimestampToDate(col 8:timestamp) -> 52:date) -> 55:timestamp, VectorUDFAdaptor(CAST( cstring1 AS TIMESTAMP)) -> 56:timestamp, VectorUDFAdaptor(CAST( substr(cstring1, 1, 1) AS TIMESTAMP))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 57:timestamp, CastLongToString(col 0:tinyint) -> 26:string, CastLongToString(col 1:smallint) -> 58:string, CastLongToString(col 2:int) -> 59:string, CastLongToString(col 3:bigint) -> 60:string, VectorUDFAdaptor(UDFToString(cfloat)) -> 61:string, VectorUDFAdaptor(UDFToString(cdouble)) -> 62:string, CastBooleanToStringViaLongToString(col 10:boolean) -> 63:string, CastLongToString(col 52:bigint)(children: LongColMultiplyLongScalar(col
  3:bigint, val 0) -> 52:bigint) -> 64:string, VectorUDFAdaptor(UDFToString(ctimestamp1)) -> 65:string, CastStringGroupToString(col 66:char(10))(children: CastStringGroupToChar(col 6:string, maxLength 10) -> 66:char(10)) -> 67:string, CastStringGroupToString(col 66:varchar(10))(children: CastStringGroupToVarChar(col 6:string, maxLength 10) -> 66:varchar(10)) -> 68:string, CastLongToFloatViaLongToDouble(col 52:int)(children: CastDoubleToLong(col 4:float) -> 52:int) -> 69:float, CastLongToDouble(col 52:int)(children: LongColMultiplyLongScalar(col 2:int, val 2) -> 52:int) -> 70:double, VectorUDFAdaptor(UDFToString(sin(cfloat)))(children: FuncSinDoubleToDouble(col 4:float) -> 71:double) -> 66:string, DoubleColAddDoubleColumn(col 71:double, col 72:double)(children: col 71:float, CastLongToDouble(col 10:boolean) -> 72:double) -> 73:double
+                          selectExpressions: CastLongToBooleanViaLongToLong(col 0:tinyint) -> 13:boolean, CastLongToBooleanViaLongToLong(col 1:smallint) -> 14:boolean, CastLongToBooleanViaLongToLong(col 2:int) -> 15:boolean, CastLongToBooleanViaLongToLong(col 3:bigint) -> 16:boolean, CastDoubleToBooleanViaDoubleToLong(col 4:float) -> 17:boolean, CastDoubleToBooleanViaDoubleToLong(col 5:double) -> 18:boolean, CastLongToBooleanViaLongToLong(col 19:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 19:bigint) -> 20:boolean, CastTimestampToBoolean(col 8:timestamp) -> 19:boolean, CastLongToBooleanViaLongToLong(col 21:bigint)(children: StringLength(col 6:string) -> 21:bigint) -> 22:boolean, CastDoubleToLong(col 4:float) -> 21:int, CastDoubleToLong(col 5:double) -> 23:int, CastTimestampToLong(col 8:timestamp) -> 24:int, CastStringToLong(col 6:string) -> 25:int, CastStringToLong(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) 
 -> 27:int, CastDoubleToLong(col 4:float) -> 28:tinyint, CastDoubleToLong(col 4:float) -> 29:smallint, CastDoubleToLong(col 4:float) -> 30:bigint, CastLongToDouble(col 0:tinyint) -> 31:double, CastLongToDouble(col 1:smallint) -> 32:double, CastLongToDouble(col 2:int) -> 33:double, CastLongToDouble(col 3:bigint) -> 34:double, CastLongToDouble(col 10:boolean) -> 35:double, CastTimestampToDouble(col 8:timestamp) -> 36:double, CastStringToDouble(col 6:string) -> 37:double, CastStringToDouble(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 38:double, CastLongToFloatViaLongToDouble(col 2:int) -> 39:float, CastMillisecondsLongToTimestamp(col 0:tinyint) -> 41:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 43:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 45:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 47:timestamp, CastDoubleToTimestamp(col 4:float) -> 48:timestamp, CastDoubleToTimestamp(col 5:double) 
 -> 49:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 51:timestamp, CastMillisecondsLongToTimestamp(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 54:timestamp, CastDateToTimestamp(col 52:date)(children: CastTimestampToDate(col 8:timestamp) -> 52:date) -> 55:timestamp, VectorUDFAdaptor(CAST( cstring1 AS TIMESTAMP)) -> 56:timestamp, VectorUDFAdaptor(CAST( substr(cstring1, 1, 1) AS TIMESTAMP))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 57:timestamp, CastLongToString(col 0:tinyint) -> 26:string, CastLongToString(col 1:smallint) -> 58:string, CastLongToString(col 2:int) -> 59:string, CastLongToString(col 3:bigint) -> 60:string, CastFloatToString(col 4:float) -> 61:string, CastDoubleToString(col 5:double) -> 62:string, CastBooleanToStringViaLongToString(col 10:boolean) -> 63:string, CastLongToString(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 64:st
 ring, VectorUDFAdaptor(UDFToString(ctimestamp1)) -> 65:string, CastStringGroupToString(col 66:char(10))(children: CastStringGroupToChar(col 6:string, maxLength 10) -> 66:char(10)) -> 67:string, CastStringGroupToString(col 66:varchar(10))(children: CastStringGroupToVarChar(col 6:string, maxLength 10) -> 66:varchar(10)) -> 68:string, CastLongToFloatViaLongToDouble(col 52:int)(children: CastDoubleToLong(col 4:float) -> 52:int) -> 69:float, CastLongToDouble(col 52:int)(children: LongColMultiplyLongScalar(col 2:int, val 2) -> 52:int) -> 70:double, CastDoubleToString(col 71:double)(children: FuncSinDoubleToDouble(col 4:float) -> 71:double) -> 66:string, DoubleColAddDoubleColumn(col 71:double, col 72:double)(children: col 71:float, CastLongToDouble(col 10:boolean) -> 72:double) -> 73:double
                       Statistics: Num rows: 6144 Data size: 16362860 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false

http://git-wip-us.apache.org/repos/asf/hive/blob/c58435bf/ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out
index 5c0a90e..22f2894 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out
@@ -670,7 +670,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -866,7 +866,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -1069,7 +1069,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -1212,7 +1212,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -1647,7 +1647,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.mapred.TextInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 4 
             Map Operator Tree:
@@ -1691,7 +1691,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -1805,7 +1805,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.mapred.TextInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 4 
             Map Operator Tree:
@@ -1849,7 +1849,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -1963,7 +1963,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.mapred.TextInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 4 
             Map Operator Tree:
@@ -1992,7 +1992,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -2106,7 +2106,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.mapred.TextInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 4 
             Map Operator Tree:
@@ -2135,7 +2135,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -2262,7 +2262,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.mapred.TextInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 4 
             Map Operator Tree:
@@ -2306,7 +2306,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -2609,7 +2609,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -2786,7 +2786,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -3406,7 +3406,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: llap
@@ -3541,7 +3541,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 5 
             Map Operator Tree:
@@ -3569,7 +3569,7 @@ STAGE PLANS:
                 inputFormatFeatureSupport: []
                 featureSupportInUse: []
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 6 
             Map Operator Tree:
@@ -4865,7 +4865,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap
@@ -5046,7 +5046,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap
@@ -5317,7 +5317,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.mapred.TextInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 3 
             Map Operator Tree:
@@ -5361,7 +5361,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap
@@ -5469,7 +5469,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.mapred.TextInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 3 
             Map Operator Tree:
@@ -5513,7 +5513,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap
@@ -6249,7 +6249,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap
@@ -6359,7 +6359,7 @@ STAGE PLANS:
                 featureSupportInUse: []
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 2 
             Map Operator Tree:
@@ -6387,7 +6387,7 @@ STAGE PLANS:
                 inputFormatFeatureSupport: []
                 featureSupportInUse: []
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Map 3 
             Map Operator Tree: