Skip to content

Commit

Permalink
Extract AbstractTestFunctions
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum committed Apr 8, 2015
1 parent 269fb6b commit 2713b4d
Show file tree
Hide file tree
Showing 17 changed files with 137 additions and 384 deletions.
Expand Up @@ -16,13 +16,15 @@
import com.facebook.presto.metadata.OperatorType;
import com.facebook.presto.operator.scalar.ScalarFunction;
import com.facebook.presto.operator.scalar.ScalarOperator;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.type.StandardTypes;
import io.airlift.slice.Slice;
import org.jcodings.specific.UTF8Encoding;
import org.joni.Option;
import org.joni.Regex;
import org.joni.Syntax;

import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.joni.constants.MetaChar.INEFFECTIVE_META_CHAR;
import static org.joni.constants.SyntaxProperties.OP_ASTERISK_ZERO_INF;
Expand Down Expand Up @@ -135,7 +137,7 @@ private static char getEscapeChar(Slice escape)
if (escapeString.length() == 1) {
return escapeString.charAt(0);
}
throw new IllegalArgumentException("escape must be empty or a single character: " + escapeString);
throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Escape must be empty or a single character");
}

private static boolean isAscii(byte[] bytes)
Expand Down
@@ -0,0 +1,84 @@
/*
* 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;

import com.facebook.presto.metadata.FunctionListBuilder;
import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.ParametricFunction;
import com.facebook.presto.spi.ErrorCodeSupplier;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.type.Type;

import java.util.List;

import static com.facebook.presto.spi.StandardErrorCode.INVALID_CAST_ARGUMENT;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static com.facebook.presto.type.UnknownType.UNKNOWN;
import static java.lang.String.format;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.fail;

public abstract class AbstractTestFunctions
{
protected final FunctionAssertions functionAssertions = new FunctionAssertions();

protected void assertFunction(String projection, Type expectedType, Object expected)
{
functionAssertions.assertFunction(projection, expectedType, expected);
}

protected void assertInvalidFunction(String projection, Type expectedType, String message)
{
try {
assertFunction(projection, expectedType, null);
fail("Expected to throw an INVALID_FUNCTION_ARGUMENT exception with message " + message);
}
catch (PrestoException e) {
assertEquals(e.getErrorCode(), INVALID_FUNCTION_ARGUMENT.toErrorCode());
assertEquals(e.getMessage(), message);
}
}

protected void assertInvalidFunction(String projection, Type expectedType, ErrorCodeSupplier expectedErrorCode)
{
try {
assertFunction(projection, expectedType, null);
fail(format("Expected to throw %s exception", expectedErrorCode.toErrorCode()));
}
catch (PrestoException e) {
assertEquals(e.getErrorCode(), expectedErrorCode.toErrorCode());
}
}

protected void assertInvalidCast(String projection)
{
try {
// type isn't necessary as the function is not valid
functionAssertions.assertFunction(projection, UNKNOWN, null);
fail("Expected to throw an INVALID_CAST_ARGUMENT exception");
}
catch (PrestoException e) {
assertEquals(e.getErrorCode(), INVALID_CAST_ARGUMENT.toErrorCode());
}
}

protected void registerScalar(Class<?> clazz)
{
Metadata metadata = functionAssertions.getMetadata();
List<ParametricFunction> functions = new FunctionListBuilder(metadata.getTypeManager())
.scalar(clazz)
.getFunctions();
metadata.getFunctionRegistry().addFunctions(functions);
}
}
Expand Up @@ -39,7 +39,6 @@
import com.facebook.presto.spi.HostAddress;
import com.facebook.presto.spi.InMemoryRecordSet;
import com.facebook.presto.spi.Page;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.RecordPageSource;
import com.facebook.presto.spi.RecordSet;
import com.facebook.presto.spi.block.Block;
Expand All @@ -63,7 +62,6 @@
import com.facebook.presto.sql.tree.QualifiedNameReference;
import com.facebook.presto.testing.LocalQueryRunner;
import com.facebook.presto.testing.MaterializedResult;
import com.facebook.presto.type.UnknownType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
Expand All @@ -87,7 +85,6 @@
import static com.facebook.presto.block.BlockAssertions.createStringsBlock;
import static com.facebook.presto.operator.scalar.FunctionAssertions.TestSplit.createNormalSplit;
import static com.facebook.presto.operator.scalar.FunctionAssertions.TestSplit.createRecordSetSplit;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_CAST_ARGUMENT;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
Expand All @@ -106,7 +103,6 @@
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;

public final class FunctionAssertions
{
Expand Down Expand Up @@ -211,23 +207,6 @@ else if (expected instanceof Slice) {
}
}

public void assertFunctionNull(String projection, Type expectedType)
{
assertNull(selectSingleValue(projection, expectedType, compiler));
}

public void assertInvalidFunction(String projection)
{
try {
// type isn't necessary as the function is not valid
assertFunction(projection, UnknownType.UNKNOWN, null);
fail();
}
catch (PrestoException e) {
assertEquals(e.getErrorCode(), INVALID_CAST_ARGUMENT.toErrorCode());
}
}

public void tryEvaluate(String expression, Type expectedType)
{
tryEvaluate(expression, expectedType, session);
Expand Down
Expand Up @@ -13,26 +13,17 @@
*/
package com.facebook.presto.operator.scalar;

import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.type.Type;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

import static com.facebook.presto.spi.StandardErrorCode.DIVISION_BY_ZERO;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;

public class TestConditions
extends AbstractTestFunctions
{
private FunctionAssertions functionAssertions;

@BeforeClass
public void setUp()
{
functionAssertions = new FunctionAssertions();
}

@Test
public void testLike()
{
Expand Down Expand Up @@ -75,6 +66,8 @@ public void testLike()
assertFunction("null not like 'monkey'", BOOLEAN, null);
assertFunction("'monkey' not like null", BOOLEAN, null);
assertFunction("'monkey' not like 'monkey' escape null", BOOLEAN, null);

assertInvalidFunction("'monkey' like 'monkey' escape 'foo'", VARCHAR, "Escape must be empty or a single character");
}

@Test
Expand All @@ -93,13 +86,6 @@ public void testDistinctFrom()
assertFunction("1 IS NOT DISTINCT FROM 1", BOOLEAN, true);
assertFunction("1 IS NOT DISTINCT FROM 2", BOOLEAN, false);
}

@Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = ".*escape must be empty or a single character.*")
public void testLikeInvalidEscape()
{
evaluate("'monkey' like 'monkey' escape 'foo'", VARCHAR);
}

@Test
public void testBetween()
{
Expand Down Expand Up @@ -166,10 +152,10 @@ public void testIn()
assertFunction("(3 not in (2, null)) is null", BOOLEAN, true);
}

@Test(expectedExceptions = PrestoException.class)
@Test
public void testInDoesNotShortCircuit()
{
evaluate("3 in (2, 4, 3, 5 / 0)", DOUBLE);
assertInvalidFunction("3 in (2, 4, 3, 5 / 0)", DOUBLE, DIVISION_BY_ZERO);
}

@Test
Expand Down Expand Up @@ -291,14 +277,4 @@ public void testSimpleCase()
DOUBLE,
33.0);
}

private void assertFunction(String projection, Type expectedType, Object expected)
{
functionAssertions.assertFunction(projection, expectedType, expected);
}

private void evaluate(String projection, Type expectedType)
{
functionAssertions.tryEvaluate(projection, expectedType);
}
}
Expand Up @@ -14,30 +14,18 @@
package com.facebook.presto.operator.scalar;

import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.type.UnknownType;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
import static com.facebook.presto.type.JsonType.JSON;
import static io.airlift.slice.Slices.utf8Slice;
import static java.lang.String.format;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.fail;

public class TestJsonFunctions
extends AbstractTestFunctions
{
private FunctionAssertions functionAssertions;

@BeforeClass
public void setUp()
{
functionAssertions = new FunctionAssertions();
}

@Test
public void testJsonArrayLength()
{
Expand Down Expand Up @@ -289,10 +277,10 @@ public void testJsonSize()
assertFunction(format("JSON_SIZE(CAST('%s' AS JSON), '%s')", "[1,2,3]", "$"), BIGINT, 3);
assertFunction(format("JSON_SIZE(null, '%s')", "$"), BIGINT, null);
assertFunction(format("JSON_SIZE(CAST('%s' AS JSON), null)", "[1,2,3]"), BIGINT, null);
assertInvalidFunction(format("JSON_SIZE('%s', '%s')", "{\"\":\"\"}", ""), "Invalid JSON path: ''");
assertInvalidFunction(format("JSON_SIZE('%s', '%s')", "{\"\":\"\"}", "."), "Invalid JSON path: '.'");
assertInvalidFunction(format("JSON_SIZE('%s', '%s')", "{\"\":\"\"}", "null"), "Invalid JSON path: 'null'");
assertInvalidFunction(format("JSON_SIZE('%s', '%s')", "{\"\":\"\"}", null), "Invalid JSON path: 'null'");
assertInvalidFunction(format("JSON_SIZE('%s', '%s')", "{\"\":\"\"}", ""), BIGINT, "Invalid JSON path: ''");
assertInvalidFunction(format("JSON_SIZE('%s', '%s')", "{\"\":\"\"}", "."), BIGINT, "Invalid JSON path: '.'");
assertInvalidFunction(format("JSON_SIZE('%s', '%s')", "{\"\":\"\"}", "null"), BIGINT, "Invalid JSON path: 'null'");
assertInvalidFunction(format("JSON_SIZE('%s', '%s')", "{\"\":\"\"}", null), BIGINT, "Invalid JSON path: 'null'");
}

@Test
Expand All @@ -317,21 +305,4 @@ public void testJsonEquality()
assertFunction("CAST('{\"p_1\": 1, \"p_2\":\"v_2\", \"p_3\":null, \"p_4\":true, \"p_5\": {\"p_1\":1}}' AS JSON) != " +
"CAST('{\"p_2\":\"v_2\", \"p_4\":true, \"p_1\": 1, \"p_3\":null, \"p_5\": {\"p_1\":1}}' AS JSON)", BOOLEAN, false);
}

private void assertFunction(String projection, Type expectedType, Object expected)
{
functionAssertions.assertFunction(projection, expectedType, expected);
}

private void assertInvalidFunction(String projection, String message)
{
try {
assertFunction(projection, UnknownType.UNKNOWN, null);
fail("Expected to throw an INVALID_FUNCTION_ARGUMENT exception with message " + message);
}
catch (PrestoException e) {
assertEquals(e.getErrorCode(), INVALID_FUNCTION_ARGUMENT.toErrorCode());
assertEquals(e.getMessage(), message);
}
}
}

0 comments on commit 2713b4d

Please sign in to comment.