Skip to content

Commit

Permalink
Add operators for distinct types
Browse files Browse the repository at this point in the history
  • Loading branch information
pranjalssh authored and highker committed Mar 8, 2022
1 parent 5c6efc7 commit 85a5aa5
Show file tree
Hide file tree
Showing 15 changed files with 830 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -367,6 +367,17 @@
import static com.facebook.presto.operator.scalar.TryCastFunction.TRY_CAST;
import static com.facebook.presto.operator.scalar.ZipFunction.ZIP_FUNCTIONS;
import static com.facebook.presto.operator.scalar.ZipWithFunction.ZIP_WITH_FUNCTION;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeBetweenOperator.DISTINCT_TYPE_BETWEEN_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeDistinctFromOperator.DISTINCT_TYPE_DISTINCT_FROM_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeEqualOperator.DISTINCT_TYPE_EQUAL_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeGreaterThanOperator.DISTINCT_TYPE_GREATER_THAN_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeGreaterThanOrEqualOperator.DISTINCT_TYPE_GREATER_THAN_OR_EQUAL_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeHashCodeOperator.DISTINCT_TYPE_HASH_CODE_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeIndeterminateOperator.DISTINCT_TYPE_INDETERMINATE_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeLessThanOperator.DISTINCT_TYPE_LESS_THAN_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeLessThanOrEqualOperator.DISTINCT_TYPE_LESS_THAN_OR_EQUAL_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeNotEqualOperator.DISTINCT_TYPE_NOT_EQUAL_OPERATOR;
import static com.facebook.presto.operator.scalar.distinct.DistinctTypeXXHash64Operator.DISTINCT_TYPE_XX_HASH_64_OPERATOR;
import static com.facebook.presto.operator.window.AggregateWindowFunction.supplier;
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_MISSING;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_USER_ERROR;
Expand Down Expand Up @@ -855,6 +866,13 @@ private List<? extends SqlFunction> getBuildInFunctions(FeaturesConfig featuresC
.scalars(LongEnumOperators.class)
.scalars(VarcharEnumOperators.class)
.functions(DISTINCT_TYPE_FROM_CAST, DISTINCT_TYPE_TO_CAST)
.functions(DISTINCT_TYPE_EQUAL_OPERATOR, DISTINCT_TYPE_NOT_EQUAL_OPERATOR)
.functions(DISTINCT_TYPE_LESS_THAN_OPERATOR, DISTINCT_TYPE_LESS_THAN_OR_EQUAL_OPERATOR)
.functions(DISTINCT_TYPE_GREATER_THAN_OPERATOR, DISTINCT_TYPE_GREATER_THAN_OR_EQUAL_OPERATOR)
.functions(DISTINCT_TYPE_BETWEEN_OPERATOR)
.function(DISTINCT_TYPE_DISTINCT_FROM_OPERATOR)
.functions(DISTINCT_TYPE_HASH_CODE_OPERATOR, DISTINCT_TYPE_XX_HASH_64_OPERATOR)
.function(DISTINCT_TYPE_INDETERMINATE_OPERATOR)
.codegenScalars(MapFilterFunction.class);

switch (featuresConfig.getRegexLibrary()) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* Licensed 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 com.facebook.presto.operator.scalar.distinct;

import com.facebook.presto.common.type.DistinctType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.SqlOperator;
import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation;
import com.facebook.presto.spi.function.FunctionHandle;
import com.google.common.collect.ImmutableList;

import java.util.Optional;

import static com.facebook.presto.common.function.OperatorType.BETWEEN;
import static com.facebook.presto.common.type.StandardTypes.BOOLEAN;
import static com.facebook.presto.common.type.StandardTypes.DISTINCT_TYPE;
import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.NullConvention.RETURN_NULL_ON_NULL;
import static com.facebook.presto.spi.function.Signature.withVariadicBound;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes;

public class DistinctTypeBetweenOperator
extends SqlOperator
{
public static final DistinctTypeBetweenOperator DISTINCT_TYPE_BETWEEN_OPERATOR = new DistinctTypeBetweenOperator();

private DistinctTypeBetweenOperator()
{
super(BETWEEN,
ImmutableList.of(withVariadicBound("T", DISTINCT_TYPE)),
ImmutableList.of(),
parseTypeSignature(BOOLEAN),
ImmutableList.of(parseTypeSignature("T"), parseTypeSignature("T"), parseTypeSignature("T")));
}

@Override
public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
DistinctType type = (DistinctType) boundVariables.getTypeVariable("T");
Type baseType = type.getBaseType();
FunctionHandle functionHandle = functionAndTypeManager.resolveOperator(BETWEEN, fromTypes(baseType, baseType, baseType));

return new BuiltInScalarFunctionImplementation(
false,
ImmutableList.of(valueTypeArgumentProperty(RETURN_NULL_ON_NULL), valueTypeArgumentProperty(RETURN_NULL_ON_NULL), valueTypeArgumentProperty(RETURN_NULL_ON_NULL)),
functionAndTypeManager.getJavaScalarFunctionImplementation(functionHandle).getMethodHandle(),
Optional.empty());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/*
* Licensed 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 com.facebook.presto.operator.scalar.distinct;

import com.facebook.presto.common.type.DistinctType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.FunctionInvoker;
import com.facebook.presto.metadata.SqlOperator;
import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation;
import com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice;
import com.facebook.presto.spi.function.FunctionHandle;
import com.facebook.presto.spi.function.InvocationConvention;
import com.google.common.collect.ImmutableList;

import java.util.Optional;

import static com.facebook.presto.common.function.OperatorType.IS_DISTINCT_FROM;
import static com.facebook.presto.common.type.StandardTypes.BOOLEAN;
import static com.facebook.presto.common.type.StandardTypes.DISTINCT_TYPE;
import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.NullConvention.BLOCK_AND_POSITION;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.ReturnPlaceConvention;
import static com.facebook.presto.spi.function.InvocationConvention.InvocationArgumentConvention.BLOCK_POSITION;
import static com.facebook.presto.spi.function.InvocationConvention.InvocationArgumentConvention.NULL_FLAG;
import static com.facebook.presto.spi.function.InvocationConvention.InvocationReturnConvention.FAIL_ON_NULL;
import static com.facebook.presto.spi.function.Signature.withVariadicBound;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes;

public class DistinctTypeDistinctFromOperator
extends SqlOperator
{
public static final DistinctTypeDistinctFromOperator DISTINCT_TYPE_DISTINCT_FROM_OPERATOR = new DistinctTypeDistinctFromOperator();

private DistinctTypeDistinctFromOperator()
{
super(IS_DISTINCT_FROM,
ImmutableList.of(withVariadicBound("T", DISTINCT_TYPE)),
ImmutableList.of(),
parseTypeSignature(BOOLEAN),
ImmutableList.of(parseTypeSignature("T"), parseTypeSignature("T")));
}

@Override
public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
DistinctType type = (DistinctType) boundVariables.getTypeVariable("T");
Type baseType = type.getBaseType();
FunctionHandle functionHandle = functionAndTypeManager.resolveOperator(IS_DISTINCT_FROM, fromTypes(baseType, baseType));
FunctionInvoker nullFlagfunctionInvoker = functionAndTypeManager.getFunctionInvokerProvider().createFunctionInvoker(
functionHandle,
Optional.of(new InvocationConvention(
ImmutableList.of(NULL_FLAG, NULL_FLAG),
FAIL_ON_NULL,
false)));
FunctionInvoker blockPositionfunctionInvoker = functionAndTypeManager.getFunctionInvokerProvider().createFunctionInvoker(
functionHandle,
Optional.of(new InvocationConvention(
ImmutableList.of(BLOCK_POSITION, BLOCK_POSITION),
FAIL_ON_NULL,
false)));

return new BuiltInScalarFunctionImplementation(
ImmutableList.of(
new ScalarFunctionImplementationChoice(
false,
ImmutableList.of(valueTypeArgumentProperty(NULL_FLAG), valueTypeArgumentProperty(NULL_FLAG)),
ReturnPlaceConvention.STACK,
nullFlagfunctionInvoker.methodHandle(),
Optional.empty()),
new ScalarFunctionImplementationChoice(
false,
ImmutableList.of(valueTypeArgumentProperty(BLOCK_AND_POSITION), valueTypeArgumentProperty(BLOCK_AND_POSITION)),
ReturnPlaceConvention.STACK,
blockPositionfunctionInvoker.methodHandle(),
Optional.empty())));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* Licensed 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 com.facebook.presto.operator.scalar.distinct;

import com.facebook.presto.common.type.DistinctType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.SqlOperator;
import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation;
import com.facebook.presto.spi.function.FunctionHandle;
import com.google.common.collect.ImmutableList;

import java.util.Optional;

import static com.facebook.presto.common.function.OperatorType.EQUAL;
import static com.facebook.presto.common.type.StandardTypes.BOOLEAN;
import static com.facebook.presto.common.type.StandardTypes.DISTINCT_TYPE;
import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.NullConvention.RETURN_NULL_ON_NULL;
import static com.facebook.presto.spi.function.Signature.withVariadicBound;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes;

public class DistinctTypeEqualOperator
extends SqlOperator
{
public static final DistinctTypeEqualOperator DISTINCT_TYPE_EQUAL_OPERATOR = new DistinctTypeEqualOperator();

private DistinctTypeEqualOperator()
{
super(EQUAL,
ImmutableList.of(withVariadicBound("T", DISTINCT_TYPE)),
ImmutableList.of(),
parseTypeSignature(BOOLEAN),
ImmutableList.of(parseTypeSignature("T"), parseTypeSignature("T")));
}

@Override
public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
DistinctType type = (DistinctType) boundVariables.getTypeVariable("T");
Type baseType = type.getBaseType();
FunctionHandle functionHandle = functionAndTypeManager.resolveOperator(EQUAL, fromTypes(baseType, baseType));

return new BuiltInScalarFunctionImplementation(
true,
ImmutableList.of(valueTypeArgumentProperty(RETURN_NULL_ON_NULL), valueTypeArgumentProperty(RETURN_NULL_ON_NULL)),
functionAndTypeManager.getJavaScalarFunctionImplementation(functionHandle).getMethodHandle(),
Optional.empty());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* Licensed 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 com.facebook.presto.operator.scalar.distinct;

import com.facebook.presto.common.type.DistinctType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.SqlOperator;
import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation;
import com.facebook.presto.spi.function.FunctionHandle;
import com.google.common.collect.ImmutableList;

import java.util.Optional;

import static com.facebook.presto.common.function.OperatorType.GREATER_THAN;
import static com.facebook.presto.common.type.StandardTypes.BOOLEAN;
import static com.facebook.presto.common.type.StandardTypes.DISTINCT_TYPE;
import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementationChoice.NullConvention.RETURN_NULL_ON_NULL;
import static com.facebook.presto.spi.function.Signature.withVariadicBound;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes;

public class DistinctTypeGreaterThanOperator
extends SqlOperator
{
public static final DistinctTypeGreaterThanOperator DISTINCT_TYPE_GREATER_THAN_OPERATOR = new DistinctTypeGreaterThanOperator();

private DistinctTypeGreaterThanOperator()
{
super(GREATER_THAN,
ImmutableList.of(withVariadicBound("T", DISTINCT_TYPE)),
ImmutableList.of(),
parseTypeSignature(BOOLEAN),
ImmutableList.of(parseTypeSignature("T"), parseTypeSignature("T")));
}

@Override
public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
DistinctType type = (DistinctType) boundVariables.getTypeVariable("T");
Type baseType = type.getBaseType();
FunctionHandle functionHandle = functionAndTypeManager.resolveOperator(GREATER_THAN, fromTypes(baseType, baseType));

return new BuiltInScalarFunctionImplementation(
false,
ImmutableList.of(valueTypeArgumentProperty(RETURN_NULL_ON_NULL), valueTypeArgumentProperty(RETURN_NULL_ON_NULL)),
functionAndTypeManager.getJavaScalarFunctionImplementation(functionHandle).getMethodHandle(),
Optional.empty());
}
}
Loading

0 comments on commit 85a5aa5

Please sign in to comment.