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 2022/05/23 17:17:42 UTC

[GitHub] [hive] asolimando opened a new pull request, #3317: Master hive 26243 ds kll sketch vectorized

asolimando opened a new pull request, #3317:
URL: https://github.com/apache/hive/pull/3317

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   We add a vectorized implementation for the `ds_kll_sketch` UDAF
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   When this UDAF is used either alone or at the side of other vectorizable functions, it will benefit from a performance speed-up.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   No.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   
   ```mvn test -o -Dtest=TestMiniLlapLocalCliDriver -Dqfile="compute_kll_sketch.q" -Dtest.output.overwrite -pl itests/qtest -Pitests```
   
   and 
   
   ```
   mvn test -Dtest=TestMiniLlapLocalCliDriver -Dqfile_regex="sketches*.q" -Dtest.output.overwrite -pl itests/qtest -Pitests
   ```


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1015228879


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimatorFactory.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+
+public class KllHistogramEstimatorFactory {
+
+  private KllHistogramEstimatorFactory() {
+    throw new AssertionError("Suppress default constructor for non instantiation");
+  }
+
+  /**
+   * This function deserializes the serialized KLL histogram estimator from a byte array.
+   * @param buf to deserialize
+   * @return KLL histogram estimator
+   */
+  public static KllHistogramEstimator getKllHistogramEstimator(byte[] buf) {
+    return new KllHistogramEstimator(KllUtils.deserializeKll(buf, 0, buf.length));
+  }
+
+  /**
+   * This function deserializes the serialized KLL histogram estimator from a byte array.
+   * @param buf to deserialize
+   * @param start start index for deserialization
+   * @param len start+len is deserialized
+   * @return KLL histogram estimator
+   */
+  public static KllHistogramEstimator getKllHistogramEstimator(byte[] buf, int start, int len) {
+    return new KllHistogramEstimator(KllUtils.deserializeKll(buf, start, len));
+  }
+
+  /**
+   * This method creates an empty histogram estimator with a KLL sketch with k=200.
+   * @return an empty histogram estimator with a KLL sketch with k=200
+   */
+  public static KllHistogramEstimator getEmptyHistogramEstimator() {

Review Comment:
   They will be re-introduced by the next PR implementing histograms, but it's true that they are not useful in the context of this PR, so it's confusing to add them now, I will defer them to the next PR



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1015227604


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class KllHistogramEstimator {

Review Comment:
   Quoting from https://github.com/apache/hive/pull/3317#issuecomment-1148553006:
   
   > I am trying to figure out a better place to put an interface plus the concrete implementation(s) of the "complex" object incapsulating the different sketches: the only examples I could find are HyperLogLog (stored in the metastore-server module) and BloomFilter/BloomKFilter (stored in the storage-api module). As of now I have followed the HyperLogLog example but you seem to disagree, storage-api seems pretty arbitrary too, so I am not sure how to proceed here, do you have any suggestions?
   
   Do you have any suggestions for a better place to store those classes?



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1318533293

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=3317)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1024960971


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class KllHistogramEstimator {

Review Comment:
   I have moved the classes to `org.apache.hadoop.hive.ql.udf.generic.histogram` as suggested, it looks indeed better, thanks!



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1005748716


##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java:
##########
@@ -4461,17 +4464,21 @@ public static ImmutablePair<VectorAggregationDesc,String> getVectorAggregationDe
       VectorizationContext vContext)
           throws HiveException {
 
-    VectorizedUDAFs annotation =
-        AnnotationUtils.getAnnotation(evaluator.getClass(), VectorizedUDAFs.class);
-    if (annotation == null) {
-      String issue =
-          "Evaluator " + evaluator.getClass().getSimpleName() + " does not have a " +
-          "vectorized UDAF annotation (aggregation: \"" + aggregationName + "\"). " +
-          "Vectorization not supported";
-      return new ImmutablePair<VectorAggregationDesc,String>(null, issue);
+    Class<? extends VectorAggregateExpression>[] vecAggrClasses;
+    if (aggregationName.equals("ds_kll_sketch")) {

Review Comment:
   I have drafted a solution using a private enumeration, please let me know if I got what you meant



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1019986489


##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java:
##########
@@ -244,7 +247,7 @@
 
 public class Vectorizer implements PhysicalPlanResolver {
 
-  protected static transient final Logger LOG = LoggerFactory.getLogger(Vectorizer.class);
+  protected static final Logger LOG = LoggerFactory.getLogger(Vectorizer.class);

Review Comment:
   why was the transient modifier removed?



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1019992084


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class KllHistogramEstimator {

Review Comment:
   maybe `org.apache.hadoop.hive.ql.udf.generic.histogram` in hive-exec?  @szlta what do you think?



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1292188348

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=3317)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1004468411


##########
vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java:
##########
@@ -1955,6 +1962,27 @@ private void generateVectorUDAFComputeBitVector(String[] tdesc) throws Exception
         className, templateString);
   }
 
+  private void generateVectorUDAFComputeKLL(String[] tdesc) throws Exception {

Review Comment:
   how is this different from `generateVectorUDAFComputeBitVector` method? could we reuse it?



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
asolimando commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1148553006

   > PR-1824 has nothing to do with datasketches; I don't know how you followed it's conventions but you might end up in trouble...because DS also has a HLL implementation...wouldn't that conflict with the existing one?
   
   HyperLogLog is one of the few examples of vectorized functions over a "complex" data type, it seemed like a good candidate to me.
   
   > note: PR-1824 named the file `VectorUDAFComputeBitVector.txt` and internally named the method `compute_bit_vector_hll` ; I think the class name should have contained the `Hll` keyword
   > 
   > I think that the file name `VectorUDAFComputeKLL.txt` is not connected at all to the `ds_kll_sketch` function its about to vectorize...and as such its a bit confusing....
   
   I agree on that, maybe `VectorUDAF_ds_kll_sketch` would be clearer.
   
   > The current implementation doesn't really look forward: I think we have 20 _sketch_ function from datasketches already exposed as inside Hive which could be vectorized; I think they are behind the same api cover...so just vectorizing the KLL one without any sight forward and taking "ideas" from the old hll codepath doesn't seem the best idea to me...
   > 
   > ```
   > grep ^ds_ ql/src/test/results/clientpositive/llap/show_functions.q.out|grep _sketch$
   > ```
   > 
   > no need to do everything in 1 patch - but this is pretty much just copy-pasting the existing hll txtfile substituted to kll here and there...so we should do that 20 times?
   
   Sketches are behind the same api for methods like `update()` and `serialize()`, but they also have differences, like the supported constructors, it seems hard to have a uniform UDAF for all of them. I have omitted the constructor for supporting the K param of Kll sketches in the current PR, but it will be needed later on (I have implemented it as suggested here: https://github.com/apache/hive/pull/1824#issuecomment-756588127), so we will have to have a specialized version of the KLL UDAF anyway, and the same for all the other 9 sketches, I don't see a way out of this.
   
   > > For instance, you seem to be suggesting to remove all helper classes/methods etc
   > > I don't think those changes neccessary in the _metastore_ for a vectorization of this function?
   > 
   > [HIVE-26221](https://issues.apache.org/jira/browse/HIVE-26221) is something which have changes - but has no real end-user accessible value - and as such I don't think its ready.
   
   [HIVE-26221](https://issues.apache.org/jira/browse/HIVE-26221) already sensibly improves range filter estimation for runtime statistics that can be used by Tez, of course exploiting statistics in query planning is the next obvious step, but I am a fan of iterative improvements, especially which such a big patch spanning few modules. Let's have this discussion offline as it is not key here, I just wanted to highlight the needs behind the current patch by linking to this ticket.
   
   I am trying to figure out a better place to put an interface plus the concrete implementation(s) of the "complex" object incapsulating the different sketches: the only examples I could find are `HyperLogLog` (stored in the `metastore-server` module) and `BloomFilter`/`BloomKFilter` (stored in the `storage-api` module). As of now I have followed the `HyperLogLog` example but you seem to disagree, `storage-api` seems pretty arbitrary too, so I am not sure how to proceed here, do you have any suggestions?


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] kgyrtkirk commented on a diff in pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r885430661


##########
standalone-metastore/pom.xml:
##########
@@ -181,6 +182,17 @@
         <artifactId>commons-lang3</artifactId>
         <version>${commons-lang3.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.datasketches</groupId>

Review Comment:
   I think this has nothing to do with this patch



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/kll/KllUtils.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.common.histogram.kll;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * KLL serialization utilities.
+ */
+public class KllUtils {

Review Comment:
   unrelated to changes?



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class KllHistogramEstimator {

Review Comment:
   remove these classes from the metastore...



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/kll/KllUtils.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.common.histogram.kll;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * KLL serialization utilities.
+ */
+public class KllUtils {
+
+  private KllUtils() {
+    throw new AssertionError("Suppress default constructor for non instantiation");
+  }
+
+  /**
+   * KLL is serialized according to what provided by data-sketches library
+   * @param out output stream to write to
+   * @param kll KLL sketch that needs to be serialized
+   * @throws IOException if an error occurs during serialization
+   */
+  public static void serializeKll(OutputStream out, KllFloatsSketch kll) throws IOException {
+    out.write(kll.toByteArray());
+  }

Review Comment:
   do we really need these methods/classes/etc? 
   
   this really just binds the class name; the method name with no good reason...no interface; what's the architectural value here I don't see?



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimatorFactory.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+
+public class KllHistogramEstimatorFactory {
+
+  private KllHistogramEstimatorFactory() {
+    throw new AssertionError("Suppress default constructor for non instantiation");
+  }
+
+  /**
+   * This function deserializes the serialized KLL histogram estimator from a byte array.
+   * @param buf to deserialize
+   * @return KLL histogram estimator
+   */
+  public static KllHistogramEstimator getKllHistogramEstimator(byte[] buf) {
+    return new KllHistogramEstimator(KllUtils.deserializeKll(buf, 0, buf.length));
+  }
+
+  /**
+   * This function deserializes the serialized KLL histogram estimator from a byte array.
+   * @param buf to deserialize
+   * @param start start index for deserialization
+   * @param len start+len is deserialized
+   * @return KLL histogram estimator
+   */
+  public static KllHistogramEstimator getKllHistogramEstimator(byte[] buf, int start, int len) {
+    return new KllHistogramEstimator(KllUtils.deserializeKll(buf, start, len));
+  }
+
+  /**
+   * This method creates an empty histogram estimator with a KLL sketch with k=200.
+   * @return an empty histogram estimator with a KLL sketch with k=200
+   */
+  public static KllHistogramEstimator getEmptyHistogramEstimator() {

Review Comment:
   there are 3 `getEmptyHistogramEstimator` -s from which only 2 is being used; and I think 1 will be too much already...do we need these at all? what's the point of introducing these???



##########
ql/src/gen/vectorization/UDAFTemplates/VectorUDAFComputeKLL.txt:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.aggregates;
+
+import org.apache.hadoop.hive.common.histogram.KllHistogramEstimator;
+import org.apache.hadoop.hive.common.histogram.KllHistogramEstimatorFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+#IF COMPLETE
+import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
+#ENDIF COMPLETE
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationDesc;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+/**
+ * Generated from template VectorUDAFComputeKLL.txt.

Review Comment:
   this seems to be the vectorized pair of `ds_kll_sketch` or not? 
   
   can we keep the naming conventions? it will be hard to follow what-is-what....without doing so



##########
ql/src/gen/vectorization/UDAFTemplates/VectorUDAFComputeKLL.txt:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.aggregates;
+
+import org.apache.hadoop.hive.common.histogram.KllHistogramEstimator;
+import org.apache.hadoop.hive.common.histogram.KllHistogramEstimatorFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+#IF COMPLETE
+import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
+#ENDIF COMPLETE
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationDesc;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+/**
+ * Generated from template VectorUDAFComputeKLL.txt.
+ */
+@Description(name = "ds_kll_sketch", value = "_FUNC_(x) "
+    + "Returns a KllFloatsSketch in a serialized form as a binary blob."
+    + " Values must be of type float.")
+public class <ClassName> extends VectorAggregateExpression {
+
+  public <ClassName>() {
+    super();
+  }
+
+  public <ClassName>(VectorAggregationDesc vecAggrDesc) {
+    super(vecAggrDesc);
+  }
+
+  @Override
+  public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+    return new Aggregation();
+  }
+
+  @Override
+  public void aggregateInput(AggregationBuffer agg, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+#IF COMPLETE
+    <InputColumnVectorType> inputColumn = (<InputColumnVectorType>) batch.cols[this.inputExpression.getOutputColumnNum()];
+#ENDIF COMPLETE
+#IF MERGING
+    BytesColumnVector inputColumn = (BytesColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+#ENDIF MERGING
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+    Aggregation myagg = (Aggregation) agg;
+
+#IF COMPLETE
+    myagg.prepare();
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        for (int i = 0; i < batchSize; i++) {
+          myagg.estimator.addToEstimator(inputColumn.vector[0]);
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            myagg.estimator.addToEstimator(inputColumn.vector[i]);
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            myagg.estimator.addToEstimator(inputColumn.vector[i]);
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          for (int i = 0; i < batchSize; i++) {
+            myagg.estimator.addToEstimator(inputColumn.vector[0]);
+          }
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int j = 0; j < batchSize; ++j) {
+            int i = batch.selected[j];
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.vector[i]);
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.vector[i]);
+            }
+          }
+        }
+      }
+    }
+#ENDIF COMPLETE
+#IF MERGING
+    if (inputColumn.isRepeating) {
+      if (!inputColumn.isNull[0] && inputColumn.length[0] > 0) {
+        myagg.prepare();
+        KllHistogramEstimator mergingKLL = KllHistogramEstimatorFactory.getKllHistogramEstimator(

Review Comment:
   do we really have to create and bind 1-by-1 vectorized implementations for the datasketches functions ?
   function classes kinda do the same stuff... like `DATA_TO_SKETCH` ; 



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1005482644


##########
standalone-metastore/metastore-server/pom.xml:
##########
@@ -44,6 +44,16 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.datasketches</groupId>
+      <artifactId>datasketches-hive</artifactId>
+      <exclusions>

Review Comment:
   That's actually a very good idea, thanks



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1012869267


##########
standalone-metastore/pom.xml:
##########
@@ -181,6 +182,17 @@
         <artifactId>commons-lang3</artifactId>
         <version>${commons-lang3.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.datasketches</groupId>

Review Comment:
   why do you need this, it's already defined in root pom DM?



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1026365965


##########
ql/src/java/org/apache/hadoop/hive/ql/udf/generic/histogram/kll/KllUtils.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.udf.generic.histogram.kll;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * KLL serialization utilities.
+ */
+public class KllUtils {
+
+  private KllUtils() {
+    throw new AssertionError("Suppress default constructor for non instantiation");
+  }
+
+  /**
+   * KLL is serialized according to what provided by data-sketches library
+   * @param out output stream to write to
+   * @param kll KLL sketch that needs to be serialized
+   * @throws IOException if an error occurs during serialization
+   */
+  public static void serializeKll(OutputStream out, KllFloatsSketch kll) throws IOException {
+    out.write(kll.toByteArray());
+  }
+
+  /**
+   * This function deserializes the serialized KLL sketch from a stream.
+   * @param in input stream to be deserialized
+   * @return KLL sketch
+   * @throws IOException if errors occur while reading the stream
+   */
+  public static KllFloatsSketch deserializeKll(InputStream in) throws IOException {

Review Comment:
   please make sure you do not have a dead code, haven't found any usage of this method



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1320859102

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=3317)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1020487128


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class KllHistogramEstimator {

Review Comment:
   I like your suggestion, the only concern I have is that `KllHistogramEstimator` is the histogram's equivalent of [`NumDistinctValueEstimator`](https://github.com/apache/hive/blob/8b48ce50684791d53dcb942958de233263cc421f/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/ndv/NumDistinctValueEstimator.java) for NDV, I thought it might be confusing to place it elsewhere, but I am fine with whatever you and @szlta suggest.



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1292574204

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=3317)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1292129938

   Rebased on master in order to take advantage of PR-[3637](https://github.com/apache/hive/pull/3637) to support KLL sketch size parameter in the constructor (otherwise we will miss vectorization opportunities when the UDAF is used in its version specifying the sketch size). Commits implementing this will come shortly.


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1320666216

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=3317)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1015277917


##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java:
##########
@@ -304,7 +327,7 @@ public enum EnabledOverride {
         nameMap.put(
             vectorizationEnabledOverride.name().toLowerCase(), vectorizationEnabledOverride);

Review Comment:
   I haven't introduced that enum but since we are improving slightly over the existing I have done that.



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1012845371


##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java:
##########
@@ -304,7 +327,7 @@ public enum EnabledOverride {
         nameMap.put(
             vectorizationEnabledOverride.name().toLowerCase(), vectorizationEnabledOverride);

Review Comment:
   could you please create toString() method that returns name().toLowerCase() and use that



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class KllHistogramEstimator {

Review Comment:
   should we keep those classes in hive-exec module?



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimatorFactory.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+
+public class KllHistogramEstimatorFactory {
+
+  private KllHistogramEstimatorFactory() {
+    throw new AssertionError("Suppress default constructor for non instantiation");
+  }
+
+  /**
+   * This function deserializes the serialized KLL histogram estimator from a byte array.
+   * @param buf to deserialize
+   * @return KLL histogram estimator
+   */
+  public static KllHistogramEstimator getKllHistogramEstimator(byte[] buf) {

Review Comment:
   is it used? 



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimatorFactory.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+
+public class KllHistogramEstimatorFactory {
+
+  private KllHistogramEstimatorFactory() {
+    throw new AssertionError("Suppress default constructor for non instantiation");
+  }
+
+  /**
+   * This function deserializes the serialized KLL histogram estimator from a byte array.
+   * @param buf to deserialize
+   * @return KLL histogram estimator
+   */
+  public static KllHistogramEstimator getKllHistogramEstimator(byte[] buf) {
+    return new KllHistogramEstimator(KllUtils.deserializeKll(buf, 0, buf.length));
+  }
+
+  /**
+   * This function deserializes the serialized KLL histogram estimator from a byte array.
+   * @param buf to deserialize
+   * @param start start index for deserialization
+   * @param len start+len is deserialized
+   * @return KLL histogram estimator
+   */
+  public static KllHistogramEstimator getKllHistogramEstimator(byte[] buf, int start, int len) {
+    return new KllHistogramEstimator(KllUtils.deserializeKll(buf, start, len));
+  }
+
+  /**
+   * This method creates an empty histogram estimator with a KLL sketch with k=200.
+   * @return an empty histogram estimator with a KLL sketch with k=200
+   */
+  public static KllHistogramEstimator getEmptyHistogramEstimator() {

Review Comment:
   i think this is still valid, please check and remove unused factory methods



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] github-actions[bot] commented on pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1207303925

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs.
   Feel free to reach out on the dev@hive.apache.org list if the patch is in need of reviews.


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] github-actions[bot] commented on pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1272176320

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs.
   Feel free to reach out on the dev@hive.apache.org list if the patch is in need of reviews.


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] szlta commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
szlta commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1024928175


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/histogram/KllHistogramEstimator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.common.histogram;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.hadoop.hive.common.histogram.kll.KllUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class KllHistogramEstimator {

Review Comment:
   `org.apache.hadoop.hive.ql.udf.generic.histogram` sounds good to me 



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1320851569

   Rebasing on master


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1293266540

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=3317)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1004476395


##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java:
##########
@@ -4461,17 +4464,21 @@ public static ImmutablePair<VectorAggregationDesc,String> getVectorAggregationDe
       VectorizationContext vContext)
           throws HiveException {
 
-    VectorizedUDAFs annotation =
-        AnnotationUtils.getAnnotation(evaluator.getClass(), VectorizedUDAFs.class);
-    if (annotation == null) {
-      String issue =
-          "Evaluator " + evaluator.getClass().getSimpleName() + " does not have a " +
-          "vectorized UDAF annotation (aggregation: \"" + aggregationName + "\"). " +
-          "Vectorization not supported";
-      return new ImmutablePair<VectorAggregationDesc,String>(null, issue);
+    Class<? extends VectorAggregateExpression>[] vecAggrClasses;
+    if (aggregationName.equals("ds_kll_sketch")) {

Review Comment:
   could we create an enum for the supportedAggregationUdfs?



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1013960895


##########
ql/src/gen/vectorization/UDAFTemplates/VectorUDAFComputeKLL.txt:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.aggregates;
+
+import org.apache.hadoop.hive.common.histogram.KllHistogramEstimator;
+import org.apache.hadoop.hive.common.histogram.KllHistogramEstimatorFactory;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+#IF COMPLETE
+import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
+#ENDIF COMPLETE
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationDesc;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+/**
+ * Generated from template VectorUDAFComputeKLL.txt.

Review Comment:
   I am not familiar with naming conventions here, but I think Zoli is saying that a more appropriate name would be VectorUDAFComputeDsKllSketch 



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1015217555


##########
standalone-metastore/pom.xml:
##########
@@ -181,6 +182,17 @@
         <artifactId>commons-lang3</artifactId>
         <version>${commons-lang3.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.datasketches</groupId>

Review Comment:
   `standalone-metastore` does not inherit from the root pom



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1020482885


##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java:
##########
@@ -244,7 +247,7 @@
 
 public class Vectorizer implements PhysicalPlanResolver {
 
-  protected static transient final Logger LOG = LoggerFactory.getLogger(Vectorizer.class);
+  protected static final Logger LOG = LoggerFactory.getLogger(Vectorizer.class);

Review Comment:
   Sonar marked it as [code smell](https://sonarcloud.io/project/issues?issues=AYJpdDZw9k-CsauE5Bp5&open=AYJpdDZw9k-CsauE5Bp5&id=apache_hive) because for non-`Serializable` classes the modifier is a no-op and it's suggested to remove it.



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1004461154


##########
standalone-metastore/metastore-server/pom.xml:
##########
@@ -44,6 +44,16 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.datasketches</groupId>
+      <artifactId>datasketches-hive</artifactId>
+      <exclusions>

Review Comment:
   could we declare the exclusions in mvn dependency management so we won't have to define them every time we use the dependency? 



-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on a diff in pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on code in PR #3317:
URL: https://github.com/apache/hive/pull/3317#discussion_r1026448025


##########
ql/src/java/org/apache/hadoop/hive/ql/udf/generic/histogram/kll/KllUtils.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.udf.generic.histogram.kll;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.datasketches.memory.Memory;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * KLL serialization utilities.
+ */
+public class KllUtils {
+
+  private KllUtils() {
+    throw new AssertionError("Suppress default constructor for non instantiation");
+  }
+
+  /**
+   * KLL is serialized according to what provided by data-sketches library
+   * @param out output stream to write to
+   * @param kll KLL sketch that needs to be serialized
+   * @throws IOException if an error occurs during serialization
+   */
+  public static void serializeKll(OutputStream out, KllFloatsSketch kll) throws IOException {
+    out.write(kll.toByteArray());
+  }
+
+  /**
+   * This function deserializes the serialized KLL sketch from a stream.
+   * @param in input stream to be deserialized
+   * @return KLL sketch
+   * @throws IOException if errors occur while reading the stream
+   */
+  public static KllFloatsSketch deserializeKll(InputStream in) throws IOException {

Review Comment:
   I have removed it and double checked the other methods, since generated code is involved my IDE is not 100% accurate, thanks for catching 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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ merged pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
deniskuzZ merged PR #3317:
URL: https://github.com/apache/hive/pull/3317


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
asolimando commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1207447641

   Please keep the PR open


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1305484261

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=3317)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=3317&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL) [4 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=3317&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=3317&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on pull request #3317: HIVE-26243: Add vectorized implementation of the 'ds_kll_sketch' UDAF

Posted by GitBox <gi...@apache.org>.
asolimando commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1320929182

   @deniskuzZ tests are green, ready to be merged when you have a moment, thanks!


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] kgyrtkirk commented on pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1142141281

   PR-1824 has nothing to do with datasketches; I don't know how you followed it's conventions but you might end up in trouble...because DS also has a HLL implementation...wouldn't that conflict with the existing one?
   
   note: PR-1824 named the file `VectorUDAFComputeBitVector.txt` and internally named the method `compute_bit_vector_hll` ; I think the class name should have contained the `Hll` keyword
   
   I think that the file name `VectorUDAFComputeKLL.txt` is not connected at all to the `ds_kll_sketch` function its about to vectorize...and as such its a bit confusing....
   
   The current implementation doesn't really look forward: I think we have 20 *sketch* function from datasketches already exposed as inside Hive which could be vectorized; I think they are behind the same api cover...so just vectorizing the KLL one without any sight forward and taking "ideas" from the old hll codepath doesn't seem the best idea to me...
   ```
   grep ^ds_ ql/src/test/results/clientpositive/llap/show_functions.q.out|grep _sketch$
   ```
   no need to do everything in 1 patch - but this is pretty much just copy-pasting the existing hll txtfile substituted to kll here and there...so we should do that 20 times?
   
   > For instance, you seem to be suggesting to remove all helper classes/methods etc
   I don't think those changes neccessary in the *metastore* for a vectorization of this function? 
   
   HIVE-26221 is something which have changes - but has no real end-user accessible value - and as such I don't think its ready.


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
asolimando commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1142042596

   Before going into the single discussions, the general answer to all the above comments boils down to "I am trying to keep consistency with what was done here for vectorizing HyperLogLog function": https://github.com/apache/hive/pull/1824/files
   
   I sense that you don't like how that PR was designed, but since they are very close in spirit, and that their code is used side by side, I thought it was important to keep them consistent.
   
   If we need to rework the current PR, they won't match anymore, unless we rework the HLL design and implementation too, and this has its own share of cons...
   
   Assuming we go for the refactoring, most of the comments are too sketchy to give appropriate guidance over an alternative design/implementation, I will need to ask you to elaborate more on them.
   
   For instance, you seem to be suggesting to remove all helper classes/methods etc. Since it does not seem feasible to inline all the code now sitting in the helper methods/classes directly in the vectorized implementation, I guess you want to place it someplace else, but I can't really decide based on your comment.
   
   For the couple of currently unused methods, I will need them in a PR depending on this one: https://issues.apache.org/jira/browse/HIVE-26221: I can remove them now and re-introduce them later, if preferable. Once again they mimic HLL methods (both naming and usage, since HLL and KLL methods will be used side by side in most places, it helps reading what's happening, see [LongColumnStatsAggregator.java#L104-L111](https://github.com/asolimando/hive/blob/master-histograms_stats_rebased/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/columnstats/aggr/LongColumnStatsAggregator.java#L104-L111), for instance).


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] asolimando commented on pull request #3317: HIVE-26243 ds kll sketch vectorized

Posted by GitBox <gi...@apache.org>.
asolimando commented on PR #3317:
URL: https://github.com/apache/hive/pull/3317#issuecomment-1272246764

   Please keep it open


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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