Skip to content

Commit

Permalink
Allow char to varchar coercion for hive tables
Browse files Browse the repository at this point in the history
  • Loading branch information
Praveen2112 committed Jun 7, 2024
1 parent 090aa31 commit 2cd89ce
Show file tree
Hide file tree
Showing 8 changed files with 134 additions and 2 deletions.
2 changes: 1 addition & 1 deletion docs/src/main/sphinx/connector/hive.md
Original file line number Diff line number Diff line change
Expand Up @@ -611,7 +611,7 @@ type conversions.
- `BOOLEAN`, `TINYINT`, `SMALLINT`, `INTEGER`, `BIGINT`, `REAL`, `DOUBLE`, `TIMESTAMP`, `DATE`, `CHAR` as well as
narrowing conversions for `VARCHAR`
* - `CHAR`
- narrowing conversions for `CHAR`
- `VARCHAR`, narrowing conversions for `CHAR`
* - `TINYINT`
- `VARCHAR`, `SMALLINT`, `INTEGER`, `BIGINT`, `DOUBLE`, `DECIMAL`
* - `SMALLINT`
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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 io.trino.plugin.hive.coercions;

import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.type.CharType;
import io.trino.spi.type.VarcharType;

import static com.google.common.base.Preconditions.checkArgument;
import static io.trino.spi.type.Varchars.truncateToLength;

public class CharToVarcharCoercer
extends TypeCoercer<CharType, VarcharType>
{
public CharToVarcharCoercer(CharType fromType, VarcharType toType)
{
super(fromType, toType);
checkArgument(toType.getBoundedLength() < fromType.getLength(), "Coercer to a wider varchar type should not be required");
}

@Override
protected void applyCoercedValue(BlockBuilder blockBuilder, Block block, int position)
{
toType.writeSlice(blockBuilder, truncateToLength(fromType.getSlice(block, position), toType.getBoundedLength()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,12 @@ public static Type createTypeFromCoercer(TypeManager typeManager, HiveType fromH
}
return Optional.empty();
}
if (fromType instanceof CharType fromCharType && toType instanceof VarcharType toVarcharType) {
if (!toVarcharType.isUnbounded() && toVarcharType.getBoundedLength() < fromCharType.getLength()) {
return Optional.of(new CharToVarcharCoercer(fromCharType, toVarcharType));
}
return Optional.empty();
}
if (fromHiveType.equals(HIVE_BYTE)) {
if (toHiveType.equals(HIVE_SHORT) || toHiveType.equals(HIVE_INT) || toHiveType.equals(HIVE_LONG)) {
return Optional.of(new IntegerNumberUpscaleCoercer<>(fromType, toType));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,8 @@ private boolean canCoerce(HiveType fromHiveType, HiveType toHiveType, HiveTimest
toHiveType.equals(HIVE_TIMESTAMP);
}
if (fromType instanceof CharType) {
return toType instanceof CharType;
return toType instanceof CharType ||
toType instanceof VarcharType;
}
if (toType instanceof VarcharType) {
return fromHiveType.equals(HIVE_BOOLEAN) ||
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* 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 io.trino.plugin.hive.coercions;

import io.trino.plugin.hive.coercions.CoercionUtils.CoercionContext;
import io.trino.spi.block.Block;
import io.trino.spi.type.Type;
import org.junit.jupiter.api.Test;

import static io.airlift.slice.Slices.utf8Slice;
import static io.trino.plugin.hive.HiveTimestampPrecision.DEFAULT_PRECISION;
import static io.trino.plugin.hive.HiveType.toHiveType;
import static io.trino.plugin.hive.coercions.CoercionUtils.createCoercer;
import static io.trino.spi.predicate.Utils.blockToNativeValue;
import static io.trino.spi.predicate.Utils.nativeValueToBlock;
import static io.trino.spi.type.CharType.createCharType;
import static io.trino.spi.type.VarcharType.createVarcharType;
import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER;
import static org.assertj.core.api.Assertions.assertThat;

public class TestCharToVarcharCoercer
{
@Test
public void testCharToSmallerVarcharCoercions()
{
assertCharToVarcharCoercion("a", createCharType(3), "a", createVarcharType(2));
assertCharToVarcharCoercion(" a", createCharType(3), " a", createVarcharType(2));
assertCharToVarcharCoercion(" aa", createCharType(4), " a", createVarcharType(2));
assertCharToVarcharCoercion("a", createCharType(3), "a", createVarcharType(2));
assertCharToVarcharCoercion("\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0", createCharType(3), "\uD83D\uDCB0", createVarcharType(1));
assertCharToVarcharCoercion("\uD83D\uDCB0\uD83D\uDCB0", createCharType(7), "\uD83D\uDCB0\uD83D\uDCB0", createVarcharType(6));
assertCharToVarcharCoercion("\uD83D\uDCB0", createCharType(3), "\uD83D\uDCB0", createVarcharType(1));
assertCharToVarcharCoercion("\uD83D\uDCB0 \uD83D\uDCB0", createCharType(7), "\uD83D\uDCB0 \uD83D\uDCB0", createVarcharType(6));
}

private static void assertCharToVarcharCoercion(String actualValue, Type fromType, String expectedValue, Type toType)
{
Block coercedBlock = createCoercer(TESTING_TYPE_MANAGER, toHiveType(fromType), toHiveType(toType), new CoercionContext(DEFAULT_PRECISION, false)).orElseThrow()
.apply(nativeValueToBlock(fromType, utf8Slice(actualValue)));
assertThat(blockToNativeValue(toType, coercedBlock))
.isEqualTo(utf8Slice(expectedValue));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,9 @@ protected void doTestHiveCoercion(HiveTableDefinition tableDefinition)
"date_to_bounded_varchar",
"char_to_bigger_char",
"char_to_smaller_char",
"char_to_string",
"char_to_bigger_varchar",
"char_to_smaller_varchar",
"string_to_char",
"varchar_to_bigger_char",
"varchar_to_smaller_char",
Expand Down Expand Up @@ -312,6 +315,9 @@ protected void insertTableRows(String tableName)
" DATE '2000-04-13', " +
" 'abc', " +
" 'abc', " +
" 'ab', " +
" 'cd', " +
" 'a', " +
" 'Bigger Value', " +
" 'Hi ', " +
" 'TrinoDB', " +
Expand Down Expand Up @@ -413,6 +419,9 @@ protected void insertTableRows(String tableName)
" DATE '1900-01-01', " +
" '\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0', " +
" '\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0', " +
" '\uD83D\uDCB0\uD83D\uDCB0', " +
" '\uD83D\uDCB0\uD83D\uDCB0', " +
" '\uD83D\uDCB0', " +
" '\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0', " +
" '\uD83D\uDCB0 \uD83D\uDCB0\uD83D\uDCB0', " +
" '\uD83D\uDCB0 \uD83D\uDCB0', " +
Expand Down Expand Up @@ -733,6 +742,15 @@ else if (isFormat.test("orc")) {
.put("char_to_smaller_char", ImmutableList.of(
"ab",
"\uD83D\uDCB0\uD83D\uDCB0"))
.put("char_to_string", ImmutableList.of(
"ab",
"\uD83D\uDCB0\uD83D\uDCB0"))
.put("char_to_bigger_varchar", ImmutableList.of(
"cd",
"\uD83D\uDCB0\uD83D\uDCB0"))
.put("char_to_smaller_varchar", ImmutableList.of(
"a",
"\uD83D\uDCB0"))
.put("timestamp_millis_to_date", ImmutableList.of(
java.sql.Date.valueOf("2022-12-31"),
java.sql.Date.valueOf("1970-01-01")))
Expand Down Expand Up @@ -1171,6 +1189,9 @@ private void assertProperAlteredTableSchema(String tableName)
row("date_to_bounded_varchar", "varchar(12)"),
row("char_to_bigger_char", "char(4)"),
row("char_to_smaller_char", "char(2)"),
row("char_to_string", "varchar"),
row("char_to_bigger_varchar", "varchar(4)"),
row("char_to_smaller_varchar", "varchar(2)"),
row("string_to_char", "char(1)"),
row("varchar_to_bigger_char", "char(6)"),
row("varchar_to_smaller_char", "char(2)"),
Expand Down Expand Up @@ -1280,6 +1301,9 @@ private void assertColumnTypes(
.put("date_to_bounded_varchar", VARCHAR)
.put("char_to_bigger_char", CHAR)
.put("char_to_smaller_char", CHAR)
.put("char_to_string", VARCHAR)
.put("char_to_bigger_varchar", VARCHAR)
.put("char_to_smaller_varchar", VARCHAR)
.put("string_to_char", CHAR)
.put("varchar_to_bigger_char", CHAR)
.put("varchar_to_smaller_char", CHAR)
Expand Down Expand Up @@ -1397,6 +1421,9 @@ private static void alterTableColumnTypes(String tableName)
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN varchar_to_special_double varchar_to_special_double double", tableName));
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN char_to_bigger_char char_to_bigger_char char(4)", tableName));
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN char_to_smaller_char char_to_smaller_char char(2)", tableName));
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN char_to_string char_to_string string", tableName));
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN char_to_bigger_varchar char_to_bigger_varchar varchar(4)", tableName));
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN char_to_smaller_varchar char_to_smaller_varchar varchar(2)", tableName));
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN string_to_char string_to_char char(1)", tableName));
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN varchar_to_bigger_char varchar_to_bigger_char char(6)", tableName));
onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN varchar_to_smaller_char varchar_to_smaller_char char(2)", tableName));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,9 @@ private static HiveTableDefinition.HiveTableDefinitionBuilder tableDefinitionBui
" date_to_bounded_varchar DATE," +
" char_to_bigger_char CHAR(3)," +
" char_to_smaller_char CHAR(3)," +
" char_to_string CHAR(3)," +
" char_to_bigger_varchar CHAR(3)," +
" char_to_smaller_varchar CHAR(3)," +
" string_to_char STRING," +
" varchar_to_bigger_char VARCHAR(4)," +
" varchar_to_smaller_char VARCHAR(20)," +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,9 @@ varchar_to_special_double VARCHAR(40),
date_to_bounded_varchar DATE,
char_to_bigger_char CHAR(3),
char_to_smaller_char CHAR(3),
char_to_string CHAR(3),
char_to_bigger_varchar CHAR(3),
char_to_smaller_varchar CHAR(3),
string_to_char STRING,
varchar_to_bigger_char VARCHAR(4),
varchar_to_smaller_char VARCHAR(20),
Expand Down

0 comments on commit 2cd89ce

Please sign in to comment.