Skip to content

Commit

Permalink
Remove usage of parseTypeSignature from Hive tests
Browse files Browse the repository at this point in the history
  • Loading branch information
martint committed Oct 14, 2019
1 parent eee8256 commit cab9af6
Show file tree
Hide file tree
Showing 2 changed files with 52 additions and 41 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,10 +31,8 @@
import io.prestosql.spi.security.Identity;
import io.prestosql.spi.security.SelectedRole;
import io.prestosql.spi.type.DateType;
import io.prestosql.spi.type.IntegerType;
import io.prestosql.spi.type.TimestampType;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeSignature;
import io.prestosql.sql.planner.Plan;
import io.prestosql.sql.planner.plan.ExchangeNode;
import io.prestosql.sql.planner.planprinter.IoPlanPrinter.ColumnConstraint;
Expand Down Expand Up @@ -104,6 +102,7 @@
import static io.prestosql.spi.type.CharType.createCharType;
import static io.prestosql.spi.type.DecimalType.createDecimalType;
import static io.prestosql.spi.type.DoubleType.DOUBLE;
import static io.prestosql.spi.type.IntegerType.INTEGER;
import static io.prestosql.spi.type.SmallintType.SMALLINT;
import static io.prestosql.spi.type.TinyintType.TINYINT;
import static io.prestosql.spi.type.VarcharType.VARCHAR;
Expand Down Expand Up @@ -253,7 +252,7 @@ public void testIoExplainWithPrimitiveTypes()
data.put("foo", new TypeAndEstimate(createUnboundedVarcharType(), new EstimatedStatsAndCost(1.0, 16.0, 16.0, 0.0, 0.0)));
data.put(Byte.toString((byte) (Byte.MAX_VALUE / 2)), new TypeAndEstimate(TINYINT, new EstimatedStatsAndCost(1.0, 10.0, 10.0, 0.0, 0.0)));
data.put(Short.toString((short) (Short.MAX_VALUE / 2)), new TypeAndEstimate(SMALLINT, new EstimatedStatsAndCost(1.0, 11.0, 11.0, 0.0, 0.0)));
data.put(Integer.toString(Integer.MAX_VALUE / 2), new TypeAndEstimate(IntegerType.INTEGER, new EstimatedStatsAndCost(1.0, 13.0, 13.0, 0.0, 0.0)));
data.put(Integer.toString(Integer.MAX_VALUE / 2), new TypeAndEstimate(INTEGER, new EstimatedStatsAndCost(1.0, 13.0, 13.0, 0.0, 0.0)));
data.put(Long.toString(Long.MAX_VALUE / 2), new TypeAndEstimate(BIGINT, new EstimatedStatsAndCost(1.0, 17.0, 17.0, 0.0, 0.0)));
data.put(Boolean.TRUE.toString(), new TypeAndEstimate(BOOLEAN, new EstimatedStatsAndCost(1.0, 10.0, 10.0, 0.0, 0.0)));
data.put("bar", new TypeAndEstimate(createCharType(3), new EstimatedStatsAndCost(1.0, 16.0, 16.0, 0.0, 0.0)));
Expand Down Expand Up @@ -1909,14 +1908,14 @@ public void testShowColumnsPartitionKey()
MaterializedResult actual = computeActual("SHOW COLUMNS FROM test_show_columns_partition_key");
Type unboundedVarchar = canonicalizeType(VARCHAR);
MaterializedResult expected = resultBuilder(getSession(), unboundedVarchar, unboundedVarchar, unboundedVarchar, unboundedVarchar)
.row("grape", canonicalizeTypeName("bigint"), "", "")
.row("orange", canonicalizeTypeName("bigint"), "", "")
.row("pear", canonicalizeTypeName("varchar(65535)"), "", "")
.row("mango", canonicalizeTypeName("integer"), "", "")
.row("lychee", canonicalizeTypeName("smallint"), "", "")
.row("kiwi", canonicalizeTypeName("tinyint"), "", "")
.row("apple", canonicalizeTypeName("varchar"), "partition key", "")
.row("pineapple", canonicalizeTypeName("varchar(65535)"), "partition key", "")
.row("grape", canonicalizeType(BIGINT).toString(), "", "")
.row("orange", canonicalizeType(BIGINT).toString(), "", "")
.row("pear", canonicalizeType(createVarcharType(65535)).toString(), "", "")
.row("mango", canonicalizeType(INTEGER).toString(), "", "")
.row("lychee", canonicalizeType(SMALLINT).toString(), "", "")
.row("kiwi", canonicalizeType(TINYINT).toString(), "", "")
.row("apple", canonicalizeType(VARCHAR).toString(), "partition key", "")
.row("pineapple", canonicalizeType(createVarcharType(65535)).toString(), "partition key", "")
.build();
assertEquals(actual, expected);
}
Expand Down Expand Up @@ -4559,12 +4558,6 @@ private Type canonicalizeType(Type type)
return TYPE_MANAGER.getType(hiveType.getTypeSignature());
}

private String canonicalizeTypeName(String type)
{
TypeSignature typeSignature = TypeSignature.parseTypeSignature(type);
return canonicalizeType(TYPE_MANAGER.getType(typeSignature)).toString();
}

private void assertColumnType(TableMetadata tableMetadata, String columnName, Type expectedType)
{
assertEquals(tableMetadata.getColumn(columnName).getType(), canonicalizeType(expectedType));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,12 @@
*/
package io.prestosql.plugin.hive;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.prestosql.spi.ErrorCode;
import io.prestosql.spi.PrestoException;
import io.prestosql.spi.type.ArrayType;
import io.prestosql.spi.type.RowType;
import io.prestosql.spi.type.Type;
import org.testng.annotations.Test;

Expand All @@ -25,7 +28,20 @@
import static io.airlift.testing.Assertions.assertContains;
import static io.prestosql.plugin.hive.HiveTestUtils.TYPE_MANAGER;
import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static io.prestosql.spi.type.BigintType.BIGINT;
import static io.prestosql.spi.type.BooleanType.BOOLEAN;
import static io.prestosql.spi.type.DateType.DATE;
import static io.prestosql.spi.type.DecimalType.createDecimalType;
import static io.prestosql.spi.type.DoubleType.DOUBLE;
import static io.prestosql.spi.type.IntegerType.INTEGER;
import static io.prestosql.spi.type.RowType.field;
import static io.prestosql.spi.type.SmallintType.SMALLINT;
import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
import static io.prestosql.spi.type.TinyintType.TINYINT;
import static io.prestosql.spi.type.TypeSignature.mapType;
import static io.prestosql.spi.type.VarbinaryType.VARBINARY;
import static io.prestosql.spi.type.VarcharType.VARCHAR;
import static io.prestosql.spi.type.VarcharType.createVarcharType;
import static java.util.Objects.requireNonNull;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.fail;
Expand All @@ -34,32 +50,33 @@ public class TestHiveTypeTranslator
{
private final TypeTranslator typeTranslator;

private final Map<String, HiveType> typeTranslationMap;
private final Map<Type, HiveType> typeTranslationMap;

public TestHiveTypeTranslator()
{
this(new HiveTypeTranslator(), ImmutableMap.of());
}

protected TestHiveTypeTranslator(TypeTranslator typeTranslator, Map<String, HiveType> overwriteTranslation)
protected TestHiveTypeTranslator(TypeTranslator typeTranslator, Map<Type, HiveType> overwriteTranslation)
{
this.typeTranslator = requireNonNull(typeTranslator, "typeTranslator is null");

ImmutableMap<String, HiveType> hiveTypeTranslationMap = ImmutableMap.<String, HiveType>builder()
.put("bigint", HiveType.HIVE_LONG)
.put("integer", HiveType.HIVE_INT)
.put("smallint", HiveType.HIVE_SHORT)
.put("tinyint", HiveType.HIVE_BYTE)
.put("double", HiveType.HIVE_DOUBLE)
.put("varchar(3)", HiveType.valueOf("varchar(3)"))
.put("varchar", HiveType.HIVE_STRING)
.put("date", HiveType.HIVE_DATE)
.put("timestamp", HiveType.HIVE_TIMESTAMP)
.put("decimal(5,3)", HiveType.valueOf("decimal(5,3)"))
.put("varbinary", HiveType.HIVE_BINARY)
.put("array(timestamp)", HiveType.valueOf("array<timestamp>"))
.put("map(boolean,varbinary)", HiveType.valueOf("map<boolean,binary>"))
.put("row(col0 integer,col1 varbinary)", HiveType.valueOf("struct<col0:int,col1:binary>"))
ImmutableMap<Type, HiveType> hiveTypeTranslationMap = ImmutableMap.<Type, HiveType>builder()
.put(BIGINT, HiveType.HIVE_LONG)
.put(INTEGER, HiveType.HIVE_INT)
.put(SMALLINT, HiveType.HIVE_SHORT)
.put(TINYINT, HiveType.HIVE_BYTE)
.put(DOUBLE, HiveType.HIVE_DOUBLE)
.put(createVarcharType(3), HiveType.valueOf("varchar(3)"))
.put(VARCHAR, HiveType.HIVE_STRING)
.put(DATE, HiveType.HIVE_DATE)
.put(TIMESTAMP, HiveType.HIVE_TIMESTAMP)
.put(createDecimalType(5, 3), HiveType.valueOf("decimal(5,3)"))
.put(VARBINARY, HiveType.HIVE_BINARY)
.put(new ArrayType(TIMESTAMP), HiveType.valueOf("array<timestamp>"))
.put(TYPE_MANAGER.getType(mapType(BOOLEAN.getTypeSignature(), VARBINARY.getTypeSignature())), HiveType.valueOf("map<boolean,binary>"))
.put(RowType.from(ImmutableList.of(field("col0", INTEGER), field("col1", VARBINARY))),
HiveType.valueOf("struct<col0:int,col1:binary>"))
.build();

typeTranslationMap = new HashMap<>();
Expand All @@ -70,22 +87,23 @@ protected TestHiveTypeTranslator(TypeTranslator typeTranslator, Map<String, Hive
@Test
public void testTypeTranslator()
{
for (Map.Entry<String, HiveType> entry : typeTranslationMap.entrySet()) {
for (Map.Entry<Type, HiveType> entry : typeTranslationMap.entrySet()) {
assertTypeTranslation(entry.getKey(), entry.getValue());
}

assertInvalidTypeTranslation("row(integer,varbinary)", NOT_SUPPORTED.toErrorCode(), "Anonymous row type is not supported in Hive. Please give each field a name: row(integer,varbinary)");
assertInvalidTypeTranslation(
RowType.anonymous(ImmutableList.of(INTEGER, VARBINARY)),
NOT_SUPPORTED.toErrorCode(),
"Anonymous row type is not supported in Hive. Please give each field a name: row(integer,varbinary)");
}

private void assertTypeTranslation(String typeName, HiveType hiveType)
private void assertTypeTranslation(Type type, HiveType hiveType)
{
Type type = TYPE_MANAGER.getType(parseTypeSignature(typeName));
assertEquals(HiveType.toHiveType(typeTranslator, type), hiveType);
}

private void assertInvalidTypeTranslation(String typeName, ErrorCode errorCode, String message)
private void assertInvalidTypeTranslation(Type type, ErrorCode errorCode, String message)
{
Type type = TYPE_MANAGER.getType(parseTypeSignature(typeName));
try {
HiveType.toHiveType(typeTranslator, type);
fail("expected exception");
Expand Down

0 comments on commit cab9af6

Please sign in to comment.