You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2020/11/07 20:05:40 UTC

[incubator-pinot] branch support_binary_ops_in_transform_functions created (now e0ff9d6)

This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a change to branch support_binary_ops_in_transform_functions
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git.


      at e0ff9d6  Adding support of logical functions AND and OR

This branch includes the following new commits:

     new e0ff9d6  Adding support of logical functions AND and OR

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[incubator-pinot] 01/01: Adding support of logical functions AND and OR

Posted by xi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a commit to branch support_binary_ops_in_transform_functions
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit e0ff9d645d17687a591926d953283b8ca75e5c3e
Author: Xiang Fu <fx...@gmail.com>
AuthorDate: Sat Nov 7 12:04:54 2020 -0800

    Adding support of logical functions AND and OR
---
 .../common/function/TransformFunctionType.java     |  3 +
 .../function/AndOperatorTransformFunction.java     | 49 +++++++++++++
 .../function/LogicalOperatorTransformFunction.java | 59 ++++++++++++++++
 .../function/OrOperatorTransformFunction.java      | 31 +++++++++
 .../function/TransformFunctionFactory.java         |  5 ++
 .../org/apache/pinot/core/util/ArrayCopyUtils.java |  6 ++
 .../function/AndOperatorTransformFunctionTest.java | 34 +++++++++
 .../LogicalOperatorTransformFunctionTest.java      | 81 ++++++++++++++++++++++
 .../function/OrOperatorTransformFunctionTest.java  | 34 +++++++++
 9 files changed, 302 insertions(+)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java b/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
index bd47121..eabd94b 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
@@ -45,6 +45,9 @@ public enum TransformFunctionType {
   LESS_THAN("less_than"),
   LESS_THAN_OR_EQUAL("less_than_or_equal"),
 
+  AND("and"),
+  OR("or"),
+
   CAST("cast"),
   CASE("case"),
   JSONEXTRACTSCALAR("jsonExtractScalar"),
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/AndOperatorTransformFunction.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/AndOperatorTransformFunction.java
new file mode 100644
index 0000000..56fb84b
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/AndOperatorTransformFunction.java
@@ -0,0 +1,49 @@
+/**
+ * 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.pinot.core.operator.transform.function;
+
+import org.apache.pinot.common.function.TransformFunctionType;
+
+
+/**
+ * The <code>AndOperatorTransformFunction</code> extends <code>LogicalOperatorTransformFunction</code> to
+ * implement the logical operator('AND' or '&&').
+ *
+ * The results are in boolean format and stored as an integer array with 1 represents true and 0 represents false.
+ *
+ * SQL Syntax:
+ *    exprA AND exprB
+ *    exprA && exprB
+ *
+ */
+public class AndOperatorTransformFunction extends LogicalOperatorTransformFunction {
+
+  @Override
+  public String getName() {
+    return TransformFunctionType.AND.getName();
+  }
+
+  @Override
+  int getLogicalFuncResult(int arg1, int arg2) {
+    if ((arg1 != 0) && (arg2 != 0)) {
+      return 1;
+    }
+    return 0;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/LogicalOperatorTransformFunction.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/LogicalOperatorTransformFunction.java
new file mode 100644
index 0000000..3e8b47c
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/LogicalOperatorTransformFunction.java
@@ -0,0 +1,59 @@
+package org.apache.pinot.core.operator.transform.function;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+import org.apache.pinot.core.common.DataSource;
+import org.apache.pinot.core.operator.blocks.ProjectionBlock;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.plan.DocIdSetPlanNode;
+import org.apache.pinot.core.util.ArrayCopyUtils;
+
+
+/**
+ * <code>LogicalOperatorTransformFunction</code> abstracts common functions for logical operators (AND, OR).
+ * The results are in boolean format and stored as an integer array with 1 represents true and 0 represents false.
+ */
+public abstract class LogicalOperatorTransformFunction extends BaseTransformFunction {
+
+  protected List<TransformFunction> _arguments;
+  protected int[] _results;
+
+  @Override
+  public void init(List<TransformFunction> arguments, Map<String, DataSource> dataSourceMap) {
+    _arguments = arguments;
+    Preconditions
+        .checkState(arguments.size() > 1, String.format("Expect more than 1 argument for logical operator [%s], found %d.", getName(), arguments.size()));
+    for (TransformFunction argument : arguments) {
+      Preconditions.checkState(
+          argument.getResultMetadata().getDataType().isNumeric() && argument.getResultMetadata().isSingleValue(), String
+              .format(
+                  "Unsupported data type for logical operator [%s] arguments, only supports single-valued number. Invalid argument: expression [%s], result type [%s]",
+                 getName(), argument.getName(), argument.getResultMetadata()));
+    }
+  }
+
+  @Override
+  public TransformResultMetadata getResultMetadata() {
+    return INT_SV_NO_DICTIONARY_METADATA;
+  }
+
+  @Override
+  public int[] transformToIntValuesSV(ProjectionBlock projectionBlock) {
+    if (_results == null) {
+      _results = new int[DocIdSetPlanNode.MAX_DOC_PER_CALL];
+    }
+    int length = projectionBlock.getNumDocs();
+    ArrayCopyUtils.copy(_arguments.get(0).transformToIntValuesSV(projectionBlock), _results, length);
+    for (int i = 1; i < _arguments.size(); i++) {
+      final TransformFunction transformFunction = _arguments.get(i);
+      int[] results = transformFunction.transformToIntValuesSV(projectionBlock);
+      for (int j = 0; j < length; j++) {
+        _results[i] = getLogicalFuncResult(_results[i], results[i]);
+      }
+    }
+    return _results;
+  }
+
+  abstract int getLogicalFuncResult(int arg1, int arg2);
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/OrOperatorTransformFunction.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/OrOperatorTransformFunction.java
new file mode 100644
index 0000000..c92b56e
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/OrOperatorTransformFunction.java
@@ -0,0 +1,31 @@
+package org.apache.pinot.core.operator.transform.function;
+
+import org.apache.pinot.common.function.TransformFunctionType;
+
+
+/**
+ * The <code>OrOperatorTransformFunction</code> extends <code>LogicalOperatorTransformFunction</code> to
+ * implement the logical operator('OR' or '||').
+ *
+ * The results are in boolean format and stored as an integer array with 1 represents true and 0 represents false.
+ *
+ * SQL Syntax:
+ *    exprA OR exprB
+ *    exprA || exprB
+ *
+ */
+public class OrOperatorTransformFunction extends LogicalOperatorTransformFunction {
+
+  @Override
+  public String getName() {
+    return TransformFunctionType.OR.getName();
+  }
+
+  @Override
+  int getLogicalFuncResult(int arg1, int arg2) {
+    if ((arg1 == 0) && (arg2 == 0)) {
+      return 0;
+    }
+    return 1;
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
index cf1685f..3d6ff94 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java
@@ -109,6 +109,11 @@ public class TransformFunctionFactory {
               GreaterThanOrEqualTransformFunction.class);
           put(canonicalize(TransformFunctionType.LESS_THAN.getName().toLowerCase()), LessThanTransformFunction.class);
           put(canonicalize(TransformFunctionType.LESS_THAN_OR_EQUAL.getName().toLowerCase()), LessThanOrEqualTransformFunction.class);
+
+          // logical functions
+          put(canonicalize(TransformFunctionType.AND.getName().toLowerCase()), AndOperatorTransformFunction.class);
+          put(canonicalize(TransformFunctionType.OR.getName().toLowerCase()), OrOperatorTransformFunction.class);
+
           // geo functions
           // geo constructors
           put(canonicalize(TransformFunctionType.ST_GEOG_FROM_TEXT.getName().toLowerCase()), StGeogFromTextFunction.class);
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/util/ArrayCopyUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/util/ArrayCopyUtils.java
index d88c02c..0df2e5c 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/util/ArrayCopyUtils.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/util/ArrayCopyUtils.java
@@ -29,6 +29,12 @@ public class ArrayCopyUtils {
   private ArrayCopyUtils() {
   }
 
+  public static void copy(int[] src, int[] dest, int length) {
+    for (int i = 0; i < length; i++) {
+      dest[i] = src[i];
+    }
+  }
+
   public static void copy(int[] src, long[] dest, int length) {
     for (int i = 0; i < length; i++) {
       dest[i] = src[i];
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/AndOperatorTransformFunctionTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/AndOperatorTransformFunctionTest.java
new file mode 100644
index 0000000..7f0e854
--- /dev/null
+++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/AndOperatorTransformFunctionTest.java
@@ -0,0 +1,34 @@
+/**
+ * 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.pinot.core.operator.transform.function;
+
+public class AndOperatorTransformFunctionTest extends LogicalOperatorTransformFunctionTest {
+  @Override
+  int getExpectedValue(boolean arg1, boolean arg2) {
+    if (arg1 && arg2) {
+      return 1;
+    }
+    return 0;
+  }
+
+  @Override
+  String getFuncName() {
+    return new AndOperatorTransformFunction().getName();
+  }
+}
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/LogicalOperatorTransformFunctionTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/LogicalOperatorTransformFunctionTest.java
new file mode 100644
index 0000000..15b13e4
--- /dev/null
+++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/LogicalOperatorTransformFunctionTest.java
@@ -0,0 +1,81 @@
+/**
+ * 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.pinot.core.operator.transform.function;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.utils.StringUtil;
+import org.apache.pinot.core.query.exception.BadQueryRequestException;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FunctionContext;
+import org.apache.pinot.core.query.request.context.utils.QueryContextConverterUtils;
+import org.testng.Assert;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+
+/**
+ * LogicalOperatorTransformFunctionTest abstracts common test methods for:
+ *     AndTransformFunctionTest
+ *     OrTransformFunctionTest
+ *
+ */
+public abstract class LogicalOperatorTransformFunctionTest extends BaseTransformFunctionTest {
+
+  abstract int getExpectedValue(boolean arg1, boolean arg2);
+
+  abstract String getFuncName();
+
+  @Test
+  public void testLogicalOperatorTransformFunction() {
+    ExpressionContext intEqualsExpr =
+        QueryContextConverterUtils.getExpression(String.format("EQUALS(%s, %d)", INT_SV_COLUMN, _intSVValues[0]));
+    ExpressionContext longEqualsExpr =
+        QueryContextConverterUtils.getExpression(String.format("EQUALS(%s, %d)", LONG_SV_COLUMN, _longSVValues[0]));
+    ExpressionContext expression = ExpressionContext.forFunction(
+        new FunctionContext(FunctionContext.Type.TRANSFORM, getFuncName(),
+            Arrays.asList(intEqualsExpr, longEqualsExpr)));
+    TransformFunction transformFunction = TransformFunctionFactory.get(expression, _dataSourceMap);
+    Assert.assertEquals(transformFunction.getName(), getFuncName().toLowerCase());
+    int[] expectedIntValues = new int[NUM_ROWS];
+    for (int i = 0; i < NUM_ROWS; i++) {
+      expectedIntValues[i] = getExpectedValue(_intSVValues[i] == _intSVValues[0], _longSVValues[i] == _longSVValues[0]);
+    }
+    testTransformFunction(transformFunction, expectedIntValues);
+  }
+
+  @Test(dataProvider = "testIllegalArguments", expectedExceptions = {BadQueryRequestException.class})
+  public void testIllegalArguments(String[] expressions) {
+    List<ExpressionContext> expressionContextList = new ArrayList<>();
+    for (int i = 0; i < expressions.length; i++) {
+      expressionContextList.add(QueryContextConverterUtils.getExpression(expressions[i]));
+    }
+    TransformFunctionFactory.get(ExpressionContext
+            .forFunction(new FunctionContext(FunctionContext.Type.TRANSFORM, getFuncName(), expressionContextList)),
+        _dataSourceMap);
+  }
+
+  @DataProvider(name = "testIllegalArguments")
+  public Object[][] testIllegalArguments() {
+    String intEqualsExpr = String.format("EQUALS(%s, %d)", INT_SV_COLUMN, _intSVValues[0]);
+    return new Object[][]{new Object[]{intEqualsExpr}, new Object[]{intEqualsExpr, STRING_SV_COLUMN}};
+  }
+}
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/OrOperatorTransformFunctionTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/OrOperatorTransformFunctionTest.java
new file mode 100644
index 0000000..a137b49
--- /dev/null
+++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/transform/function/OrOperatorTransformFunctionTest.java
@@ -0,0 +1,34 @@
+/**
+ * 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.pinot.core.operator.transform.function;
+
+public class OrOperatorTransformFunctionTest extends LogicalOperatorTransformFunctionTest {
+  @Override
+  int getExpectedValue(boolean arg1, boolean arg2) {
+    if (arg1 || arg2) {
+      return 1;
+    }
+    return 0;
+  }
+
+  @Override
+  String getFuncName() {
+    return new OrOperatorTransformFunction().getName();
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org