diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/BuiltInTypeAndFunctionNamespaceManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/BuiltInTypeAndFunctionNamespaceManager.java index cd2a85455208b..54d3ed8961277 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/BuiltInTypeAndFunctionNamespaceManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/BuiltInTypeAndFunctionNamespaceManager.java @@ -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; @@ -855,6 +866,13 @@ private List 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()) { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeBetweenOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeBetweenOperator.java new file mode 100644 index 0000000000000..0631223758056 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeBetweenOperator.java @@ -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()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeDistinctFromOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeDistinctFromOperator.java new file mode 100644 index 0000000000000..48b27f4295631 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeDistinctFromOperator.java @@ -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()))); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeEqualOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeEqualOperator.java new file mode 100644 index 0000000000000..be0a2d5a6e047 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeEqualOperator.java @@ -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()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeGreaterThanOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeGreaterThanOperator.java new file mode 100644 index 0000000000000..b86b342a46efe --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeGreaterThanOperator.java @@ -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()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeGreaterThanOrEqualOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeGreaterThanOrEqualOperator.java new file mode 100644 index 0000000000000..a8d1cb563b5c8 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeGreaterThanOrEqualOperator.java @@ -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_OR_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 DistinctTypeGreaterThanOrEqualOperator + extends SqlOperator +{ + public static final DistinctTypeGreaterThanOrEqualOperator DISTINCT_TYPE_GREATER_THAN_OR_EQUAL_OPERATOR = new DistinctTypeGreaterThanOrEqualOperator(); + + private DistinctTypeGreaterThanOrEqualOperator() + { + super(GREATER_THAN_OR_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(GREATER_THAN_OR_EQUAL, 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()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeHashCodeOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeHashCodeOperator.java new file mode 100644 index 0000000000000..b26817c9c03bd --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeHashCodeOperator.java @@ -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.HASH_CODE; +import static com.facebook.presto.common.type.StandardTypes.BIGINT; +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 DistinctTypeHashCodeOperator + extends SqlOperator +{ + public static final DistinctTypeHashCodeOperator DISTINCT_TYPE_HASH_CODE_OPERATOR = new DistinctTypeHashCodeOperator(); + + private DistinctTypeHashCodeOperator() + { + super(HASH_CODE, + ImmutableList.of(withVariadicBound("T", DISTINCT_TYPE)), + ImmutableList.of(), + parseTypeSignature(BIGINT), + ImmutableList.of(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(HASH_CODE, fromTypes(baseType)); + + return new BuiltInScalarFunctionImplementation( + false, + ImmutableList.of(valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), + functionAndTypeManager.getJavaScalarFunctionImplementation(functionHandle).getMethodHandle(), + Optional.empty()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeIndeterminateOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeIndeterminateOperator.java new file mode 100644 index 0000000000000..5e3de9af668a5 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeIndeterminateOperator.java @@ -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.INDETERMINATE; +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.USE_NULL_FLAG; +import static com.facebook.presto.spi.function.Signature.withVariadicBound; +import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes; + +public class DistinctTypeIndeterminateOperator + extends SqlOperator +{ + public static final DistinctTypeIndeterminateOperator DISTINCT_TYPE_INDETERMINATE_OPERATOR = new DistinctTypeIndeterminateOperator(); + + private DistinctTypeIndeterminateOperator() + { + super(INDETERMINATE, + ImmutableList.of(withVariadicBound("T", DISTINCT_TYPE)), + ImmutableList.of(), + parseTypeSignature(BOOLEAN), + ImmutableList.of(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(INDETERMINATE, fromTypes(baseType)); + + return new BuiltInScalarFunctionImplementation( + false, + ImmutableList.of(valueTypeArgumentProperty(USE_NULL_FLAG)), + functionAndTypeManager.getJavaScalarFunctionImplementation(functionHandle).getMethodHandle(), + Optional.empty()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeLessThanOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeLessThanOperator.java new file mode 100644 index 0000000000000..cc5ce2f0b414f --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeLessThanOperator.java @@ -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.LESS_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 DistinctTypeLessThanOperator + extends SqlOperator +{ + public static final DistinctTypeLessThanOperator DISTINCT_TYPE_LESS_THAN_OPERATOR = new DistinctTypeLessThanOperator(); + + private DistinctTypeLessThanOperator() + { + super(LESS_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(LESS_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()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeLessThanOrEqualOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeLessThanOrEqualOperator.java new file mode 100644 index 0000000000000..a2bed8bb6dab7 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeLessThanOrEqualOperator.java @@ -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.LESS_THAN_OR_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 DistinctTypeLessThanOrEqualOperator + extends SqlOperator +{ + public static final DistinctTypeLessThanOrEqualOperator DISTINCT_TYPE_LESS_THAN_OR_EQUAL_OPERATOR = new DistinctTypeLessThanOrEqualOperator(); + + private DistinctTypeLessThanOrEqualOperator() + { + super(LESS_THAN_OR_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(LESS_THAN_OR_EQUAL, 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()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeNotEqualOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeNotEqualOperator.java new file mode 100644 index 0000000000000..d9a1e7f28ee9b --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeNotEqualOperator.java @@ -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.NOT_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 DistinctTypeNotEqualOperator + extends SqlOperator +{ + public static final DistinctTypeNotEqualOperator DISTINCT_TYPE_NOT_EQUAL_OPERATOR = new DistinctTypeNotEqualOperator(); + + private DistinctTypeNotEqualOperator() + { + super(NOT_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(NOT_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()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeXXHash64Operator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeXXHash64Operator.java new file mode 100644 index 0000000000000..9e377f8d7a2a6 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/distinct/DistinctTypeXXHash64Operator.java @@ -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.XX_HASH_64; +import static com.facebook.presto.common.type.StandardTypes.BIGINT; +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 DistinctTypeXXHash64Operator + extends SqlOperator +{ + public static final DistinctTypeXXHash64Operator DISTINCT_TYPE_XX_HASH_64_OPERATOR = new DistinctTypeXXHash64Operator(); + + private DistinctTypeXXHash64Operator() + { + super(XX_HASH_64, + ImmutableList.of(withVariadicBound("T", DISTINCT_TYPE)), + ImmutableList.of(), + parseTypeSignature(BIGINT), + ImmutableList.of(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(XX_HASH_64, fromTypes(baseType)); + + return new BuiltInScalarFunctionImplementation( + false, + ImmutableList.of(valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), + functionAndTypeManager.getJavaScalarFunctionImplementation(functionHandle).getMethodHandle(), + Optional.empty()); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java index 7f6de42047d22..c99291db2b484 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java @@ -20,6 +20,7 @@ import com.facebook.presto.common.type.CharType; import com.facebook.presto.common.type.DecimalParseResult; import com.facebook.presto.common.type.Decimals; +import com.facebook.presto.common.type.DistinctType; import com.facebook.presto.common.type.FunctionType; import com.facebook.presto.common.type.MapType; import com.facebook.presto.common.type.RowType; @@ -463,6 +464,9 @@ protected Type visitDereferenceExpression(DereferenceExpression node, StackableA if (((baseType instanceof TypeWithName) && ((TypeWithName) baseType).getType() instanceof RowType)) { baseType = ((TypeWithName) baseType).getType(); } + if (baseType instanceof DistinctType) { + baseType = ((DistinctType) baseType).getBaseType(); + } if (!(baseType instanceof RowType)) { throw new SemanticException(TYPE_MISMATCH, node.getBase(), "Expression %s is not of type ROW", node.getBase()); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java index 0c82c03ac1280..111b2d8c5fb69 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java @@ -18,6 +18,7 @@ import com.facebook.presto.common.type.CharType; import com.facebook.presto.common.type.DecimalParseResult; import com.facebook.presto.common.type.Decimals; +import com.facebook.presto.common.type.DistinctType; import com.facebook.presto.common.type.RowType; import com.facebook.presto.common.type.RowType.Field; import com.facebook.presto.common.type.Type; @@ -612,6 +613,10 @@ protected RowExpression visitDereferenceExpression(DereferenceExpression node, V if (baseType instanceof TypeWithName) { baseType = ((TypeWithName) baseType).getType(); } + + if (baseType instanceof DistinctType) { + baseType = ((DistinctType) baseType).getBaseType(); + } RowType rowType = (RowType) baseType; String fieldName = node.getField().getValue(); List fields = rowType.getFields(); diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestDistinctType.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestDistinctType.java index 210a920d9e259..3d1b122ab7163 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestDistinctType.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestDistinctType.java @@ -201,6 +201,88 @@ public void testCasts() assertSingleResultFromValues("SELECT ARRAY_INTERSECT(ARRAY[ARRAY[CAST(x AS test.dt.int30)]], ARRAY[ARRAY[CAST(1 AS test.dt.int11)]])", "1", ImmutableList.of(ImmutableList.of(1))); } + @Test + public void testDistinctFrom() + { + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) IS DISTINCT FROM CAST(x as test.dt.int00)", "1", false); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) IS DISTINCT FROM CAST(x as test.dt.int00)", "2", true); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) IS DISTINCT FROM CAST(x as test.dt.int30)", "1", false); + assertSingleResultFromValues("SELECT NULL IS DISTINCT FROM CAST(x as test.dt.int00)", "1", true); + assertSingleResultFromValues("SELECT CAST(x as test.dt.int00) IS DISTINCT FROM NULL", "1", true); + + assertSingleResultFromValues("SELECT ARRAY[ROW(CAST(1 as test.dt.int00))] IS DISTINCT FROM ARRAY[ROW(CAST(x as test.dt.int00))]", "1", false); + assertQueryFails("SELECT CAST(1 as test.dt.int00) IS DISTINCT FROM CAST(1 as test.dt.int_alt)", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) IS DISTINCT FROM 1", ".*cannot be applied to test.dt.int00.*"); + } + + @Test + public void testHash() + { + assertQueryResultUnordered("SELECT DISTINCT x FROM (VALUES CAST(1 as test.dt.int00), CAST(1 as test.dt.int00), CAST(2 as test.dt.int00)) t(x)", ImmutableList.of(ImmutableList.of(1), ImmutableList.of(2))); + assertSingleResult("SELECT APPROX_DISTINCT(x) FROM (VALUES CAST(1 as test.dt.int00), CAST(1 as test.dt.int00), CAST(2 as test.dt.int00)) t(x)", 2L); + assertQueryResultUnordered("SELECT x FROM (VALUES CAST(1 as test.dt.int00), CAST(1 as test.dt.int00), CAST(2 as test.dt.int00)) t(x) GROUP BY x", ImmutableList.of(ImmutableList.of(1), ImmutableList.of(2))); + + assertQueryResultUnordered("SELECT DISTINCT x FROM (VALUES ARRAY[CAST(1 as test.dt.int00)], ARRAY[CAST(1 as test.dt.int00)], ARRAY[CAST(2 as test.dt.int00)]) t(x)", ImmutableList.of(ImmutableList.of(ImmutableList.of(1)), ImmutableList.of(ImmutableList.of(2)))); + } + + @Test + public void testComparison() + { + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) = CAST(x as test.dt.int00)", "1", true); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) = CAST(x as test.dt.int11)", "1", true); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) = CAST(x as test.dt.int11)", "2", false); + assertSingleResultFromValues("SELECT CAST(x as test.dt.int00) = NULL", "1", null); + assertSingleResultFromValues("SELECT ARRAY[ROW(CAST(1 as test.dt.int30))] = ARRAY[ROW(CAST(x as test.dt.int21))]", "1", true); + + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) <> CAST(x as test.dt.int00)", "1", false); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) <> CAST(x as test.dt.int11)", "1", false); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) <> CAST(x as test.dt.int11)", "2", true); + assertSingleResultFromValues("SELECT CAST(x as test.dt.int00) <> NULL", "1", null); + assertSingleResultFromValues("SELECT ARRAY[ROW(CAST(1 as test.dt.int30))] <> ARRAY[ROW(CAST(x as test.dt.int21))]", "1", false); + + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) < CAST(x as test.dt.int00)", "1", false); + assertSingleResultFromValues("SELECT CAST(2 as test.dt.int30) < CAST(x as test.dt.int11)", "1", false); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) < CAST(x as test.dt.int11)", "2", true); + assertSingleResultFromValues("SELECT CAST(x as test.dt.int00) < NULL", "1", null); + assertSingleResultFromValues("SELECT ARRAY[ROW(CAST(1 as test.dt.int30))] < ARRAY[ROW(CAST(x as test.dt.int21))]", "2", true); + + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) <= CAST(x as test.dt.int00)", "1", true); + assertSingleResultFromValues("SELECT CAST(2 as test.dt.int30) <= CAST(x as test.dt.int11)", "1", false); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) <= CAST(x as test.dt.int11)", "2", true); + assertSingleResultFromValues("SELECT CAST(x as test.dt.int00) <= NULL", "1", null); + assertSingleResultFromValues("SELECT ARRAY[ROW(CAST(1 as test.dt.int30))] <= ARRAY[ROW(CAST(x as test.dt.int21))]", "1", true); + + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) > CAST(x as test.dt.int00)", "1", false); + assertSingleResultFromValues("SELECT CAST(2 as test.dt.int30) > CAST(x as test.dt.int11)", "1", true); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) > CAST(x as test.dt.int11)", "2", false); + assertSingleResultFromValues("SELECT CAST(x as test.dt.int00) > NULL", "1", null); + assertSingleResultFromValues("SELECT ARRAY[ROW(CAST(1 as test.dt.int30))] > ARRAY[ROW(CAST(x as test.dt.int21))]", "1", false); + + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int00) >= CAST(x as test.dt.int00)", "1", true); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) >= CAST(x as test.dt.int11)", "1", true); + assertSingleResultFromValues("SELECT CAST(1 as test.dt.int30) >= CAST(x as test.dt.int11)", "2", false); + assertSingleResultFromValues("SELECT CAST(x as test.dt.int00) >= NULL", "1", null); + assertSingleResultFromValues("SELECT ARRAY[ROW(CAST(2 as test.dt.int30))] >= ARRAY[ROW(CAST(x as test.dt.int21))]", "1", true); + + assertSingleResultFromValues("SELECT CAST(x as test.dt.int00) BETWEEN CAST(1 as test.dt.int11) AND CAST(3 as test.dt.int30)", "2", true); + assertSingleResultFromValues("SELECT NULL BETWEEN CAST(x as test.dt.int11) AND CAST(3 as test.dt.int30)", "2", null); + + assertQueryFails("SELECT CAST(1 as test.dt.int00) = CAST(1 as test.dt.int_alt)", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) = 1", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) <> CAST(1 as test.dt.int_alt)", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) <> 1", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) < CAST(1 as test.dt.int_alt)", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) < 1", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) <= CAST(1 as test.dt.int_alt)", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) <= 1", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) > CAST(1 as test.dt.int_alt)", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) > 1", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) >= CAST(1 as test.dt.int_alt)", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT CAST(1 as test.dt.int00) >= 1", ".*cannot be applied to test.dt.int00.*"); + assertQueryFails("SELECT 2 BETWEEN CAST(1 as test.dt.int11) AND CAST(3 as test.dt.int30)", ".*Cannot check if integer is BETWEEN.*"); + assertQueryFails("SELECT CAST(2 as test.dt.int_alt) BETWEEN CAST(1 as test.dt.int11) AND CAST(3 as test.dt.int30)", ".*Cannot check if test.dt.int_alt.*"); + } + private void assertQueryResultUnordered(@Language("SQL") String query, List> expectedRows) { MaterializedResult rows = computeActual(query);