You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/01/15 03:30:13 UTC

[GitHub] [hive] ramesh0201 opened a new pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

ramesh0201 opened a new pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] jdere commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
jdere commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#discussion_r371534428
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDecimal64.java
 ##########
 @@ -0,0 +1,45 @@
+/*
+ * 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.Decimal64ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+
+/**
+ * To be used to cast long and boolean to decimal.
+ * This works for boolean too because boolean is encoded as 0
+ * for false and 1 for true.
+ */
+public class CastLongToDecimal64 extends FuncLongToDecimal64 {
 
 Review comment:
   This is currently the only concrete class for FuncLongToDecimal64 .. is it expected that there are going to be more in the future (which would be the point of making FuncLongToDecimal64 abstract?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ramesh0201 commented on issue #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
ramesh0201 commented on issue #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#issuecomment-579036627
 
 
   Whenever we have an expression like `long column <compare/arithmetic operator> decimal column`, we always cast long to decimal even if the decimal column can be represented as decimal64. This prevents us from using Deicmal64 class of operations that will be faster than the equivalent Decimal class of operations. This Jira will implements cast from long to decimal64 that can help in more optimizations in the decimal64 operations.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ramesh0201 commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
ramesh0201 commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#discussion_r371567891
 
 

 ##########
 File path: ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out
 ##########
 @@ -369,7 +369,7 @@ STAGE PLANS:
                     includeColumns: [0, 1, 2, 3, 4, 5, 6, 7, 9, 10, 11]
                     dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [double, double, double, decimal(22,3), decimal(13,3), double, double, double, double, double, double, double, double]
+                    scratchColumnTypeNames: [double, double, double, decimal(22,3), decimal(13,3)/DECIMAL_64, double, double, double, double, double, double, double, double]
 
 Review comment:
   Yes, because we can safely store a decimal with (13,3) by scaling them up and storing in long vectors and tracking the scale. We do this until precision+scale is <=18.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ramesh0201 commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
ramesh0201 commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#discussion_r371568368
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDecimal64.java
 ##########
 @@ -0,0 +1,45 @@
+/*
+ * 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.Decimal64ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+
+/**
+ * To be used to cast long and boolean to decimal.
+ * This works for boolean too because boolean is encoded as 0
+ * for false and 1 for true.
+ */
+public class CastLongToDecimal64 extends FuncLongToDecimal64 {
 
 Review comment:
   Thank you, I have addressed this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ramesh0201 commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
ramesh0201 commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#discussion_r371568274
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal64.java
 ##########
 @@ -0,0 +1,185 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.hadoop.hive.ql.exec.vector.Decimal64ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * This is a superclass for unary long functions and expressions returning decimals that
+ * operate directly on the input and set the output.
+ */
+@VectorizedExpressionsSupportDecimal64()
+public abstract class FuncLongToDecimal64 extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+  private final int inputColumn;
+
+  private static final long[] powerOfTenTable = {
+    1L,                   // 0
+    10L,
+    100L,
+    1_000L,
+    10_000L,
+    100_000L,
+    1_000_000L,
+    10_000_000L,
+    100_000_000L,           // 8
+    1_000_000_000L,
+    10_000_000_000L,
+    100_000_000_000L,
+    1_000_000_000_000L,
+    10_000_000_000_000L,
+    100_000_000_000_000L,
+    1_000_000_000_000_000L,
+    10_000_000_000_000_000L,   // 16
+    100_000_000_000_000_000L,
+    1_000_000_000_000_000_000L, // 18
+  };
+
+  public FuncLongToDecimal64(int inputColumn, int outputColumnNum) {
+    super(outputColumnNum);
+    this.inputColumn = inputColumn;
+  }
+
+  public FuncLongToDecimal64() {
+    super();
+
+    // Dummy final assignments.
+    inputColumn = -1;
+  }
+
+  abstract protected void func(Decimal64ColumnVector outputColVector, LongColumnVector inputColVector, int i,
+      long scaleFactor);
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[inputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    Decimal64ColumnVector outputColVector = (Decimal64ColumnVector) batch.cols[outputColumnNum];
+    int outputScale = ((DecimalTypeInfo) outputTypeInfo).scale();
+    final long scaleFactor = powerOfTenTable[outputScale];
 
 Review comment:
   Yes, before we create cast expression, we verify if the output.precision + output.scale <= 18(to avoid intermediate overflow) and hence implicitly outputscale will be less than or equal to 18.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] jdere commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
jdere commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#discussion_r371539542
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal64.java
 ##########
 @@ -0,0 +1,185 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.hadoop.hive.ql.exec.vector.Decimal64ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * This is a superclass for unary long functions and expressions returning decimals that
+ * operate directly on the input and set the output.
+ */
+@VectorizedExpressionsSupportDecimal64()
+public abstract class FuncLongToDecimal64 extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+  private final int inputColumn;
+
+  private static final long[] powerOfTenTable = {
+    1L,                   // 0
+    10L,
+    100L,
+    1_000L,
+    10_000L,
+    100_000L,
+    1_000_000L,
+    10_000_000L,
+    100_000_000L,           // 8
+    1_000_000_000L,
+    10_000_000_000L,
+    100_000_000_000L,
+    1_000_000_000_000L,
+    10_000_000_000_000L,
+    100_000_000_000_000L,
+    1_000_000_000_000_000L,
+    10_000_000_000_000_000L,   // 16
+    100_000_000_000_000_000L,
+    1_000_000_000_000_000_000L, // 18
+  };
+
+  public FuncLongToDecimal64(int inputColumn, int outputColumnNum) {
+    super(outputColumnNum);
+    this.inputColumn = inputColumn;
+  }
+
+  public FuncLongToDecimal64() {
+    super();
+
+    // Dummy final assignments.
+    inputColumn = -1;
+  }
+
+  abstract protected void func(Decimal64ColumnVector outputColVector, LongColumnVector inputColVector, int i,
+      long scaleFactor);
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[inputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    Decimal64ColumnVector outputColVector = (Decimal64ColumnVector) batch.cols[outputColumnNum];
+    int outputScale = ((DecimalTypeInfo) outputTypeInfo).scale();
+    final long scaleFactor = powerOfTenTable[outputScale];
 
 Review comment:
   Is outputScale always guaranteed to be between [0, 18]?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] jdere commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
jdere commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#discussion_r371537293
 
 

 ##########
 File path: ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out
 ##########
 @@ -369,7 +369,7 @@ STAGE PLANS:
                     includeColumns: [0, 1, 2, 3, 4, 5, 6, 7, 9, 10, 11]
                     dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [double, double, double, decimal(22,3), decimal(13,3), double, double, double, double, double, double, double, double]
+                    scratchColumnTypeNames: [double, double, double, decimal(22,3), decimal(13,3)/DECIMAL_64, double, double, double, double, double, double, double, double]
 
 Review comment:
   any idea how this shows up as decimal(13,3)/DECIMAL_64 - would the decimal(13,3) part also be converted to decimal64?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ramesh0201 commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
ramesh0201 commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#discussion_r371568394
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal64.java
 ##########
 @@ -0,0 +1,185 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.hadoop.hive.ql.exec.vector.Decimal64ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * This is a superclass for unary long functions and expressions returning decimals that
+ * operate directly on the input and set the output.
+ */
+@VectorizedExpressionsSupportDecimal64()
+public abstract class FuncLongToDecimal64 extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+  private final int inputColumn;
+
+  private static final long[] powerOfTenTable = {
+    1L,                   // 0
+    10L,
+    100L,
+    1_000L,
+    10_000L,
+    100_000L,
+    1_000_000L,
+    10_000_000L,
+    100_000_000L,           // 8
+    1_000_000_000L,
+    10_000_000_000L,
+    100_000_000_000L,
+    1_000_000_000_000L,
+    10_000_000_000_000L,
+    100_000_000_000_000L,
+    1_000_000_000_000_000L,
+    10_000_000_000_000_000L,   // 16
+    100_000_000_000_000_000L,
+    1_000_000_000_000_000_000L, // 18
+  };
+
+  public FuncLongToDecimal64(int inputColumn, int outputColumnNum) {
+    super(outputColumnNum);
+    this.inputColumn = inputColumn;
+  }
+
+  public FuncLongToDecimal64() {
+    super();
+
+    // Dummy final assignments.
+    inputColumn = -1;
+  }
+
+  abstract protected void func(Decimal64ColumnVector outputColVector, LongColumnVector inputColVector, int i,
 
 Review comment:
   Thank you, I have addressed this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] jdere commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization

Posted by GitBox <gi...@apache.org>.
jdere commented on a change in pull request #880: HIVE-22515 Support cast to decimal64 in Vectorization
URL: https://github.com/apache/hive/pull/880#discussion_r371533452
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal64.java
 ##########
 @@ -0,0 +1,185 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.hadoop.hive.ql.exec.vector.Decimal64ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * This is a superclass for unary long functions and expressions returning decimals that
+ * operate directly on the input and set the output.
+ */
+@VectorizedExpressionsSupportDecimal64()
+public abstract class FuncLongToDecimal64 extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+  private final int inputColumn;
+
+  private static final long[] powerOfTenTable = {
+    1L,                   // 0
+    10L,
+    100L,
+    1_000L,
+    10_000L,
+    100_000L,
+    1_000_000L,
+    10_000_000L,
+    100_000_000L,           // 8
+    1_000_000_000L,
+    10_000_000_000L,
+    100_000_000_000L,
+    1_000_000_000_000L,
+    10_000_000_000_000L,
+    100_000_000_000_000L,
+    1_000_000_000_000_000L,
+    10_000_000_000_000_000L,   // 16
+    100_000_000_000_000_000L,
+    1_000_000_000_000_000_000L, // 18
+  };
+
+  public FuncLongToDecimal64(int inputColumn, int outputColumnNum) {
+    super(outputColumnNum);
+    this.inputColumn = inputColumn;
+  }
+
+  public FuncLongToDecimal64() {
+    super();
+
+    // Dummy final assignments.
+    inputColumn = -1;
+  }
+
+  abstract protected void func(Decimal64ColumnVector outputColVector, LongColumnVector inputColVector, int i,
 
 Review comment:
   Can you use a slightly more descriptive method name here and describe what it's for?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org